diff --git a/CHANGES.txt b/CHANGES.txt index e2d1cdb5919..a53557ae395 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -5,6 +5,8 @@ Hbase Change Log Jim Kellerman) (API change for filters) HBASE-601 Just remove deprecated methods in HTable; 0.2 is not backward compatible anyways + HBASE-82 Row keys should be array of bytes + HBASE-76 Purge servers of Text (Done as part of HBASE-82 commit). BUG FIXES HBASE-574 HBase does not load hadoop native libs (Rong-En Fan via Stack) @@ -28,6 +30,9 @@ Hbase Change Log HBASE-614 Retiring regions is not used; exploit or remove HBASE-538 Improve exceptions that come out on client-side HBASE-569 DemoClient.php (Jim R. Wilson via Stack) + HBASE-522 Where new Text(string) might be used in client side method calls, + add an overload that takes string (Done as part of HBASE-82) + HBASE-570 Remove HQL unit test (Done as part of HBASE-82 commit). Release 0.1.2 - 05/13/2008 diff --git a/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java b/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java index b524fd8ccc2..efaa1b69024 100644 --- a/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java +++ b/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java @@ -22,14 +22,11 @@ package org.apache.hadoop.hbase; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.util.regex.Matcher; -import java.util.regex.Pattern; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableComparable; -import org.apache.hadoop.hbase.io.TextSequence; - /** * An HColumnDescriptor contains information about a column family such as the * number of versions, compression settings, etc. @@ -40,12 +37,11 @@ import org.apache.hadoop.hbase.io.TextSequence; * deleted when the column is deleted. */ public class HColumnDescriptor implements WritableComparable { - // For future backward compatibility - private static final byte COLUMN_DESCRIPTOR_VERSION = (byte)3; - - /** Legal family names can only contain 'word characters' and end in a colon. */ - public static final Pattern LEGAL_FAMILY_NAME = Pattern.compile("\\w+:"); + + // Version 3 was when column names becaome byte arrays and when we picked up + // Time-to-live feature. Version 4 was when we moved to byte arrays, HBASE-82. + private static final byte COLUMN_DESCRIPTOR_VERSION = (byte)4; /** * The type of compression. @@ -65,22 +61,22 @@ public class HColumnDescriptor implements WritableComparable { */ public static final CompressionType DEFAULT_COMPRESSION_TYPE = CompressionType.NONE; - + /** * Default number of versions of a record to keep. */ public static final int DEFAULT_N_VERSIONS = 3; - + /** * Default setting for whether to serve from memory or not. */ public static final boolean DEFAULT_IN_MEMORY = false; - + /** * Default setting for whether to use a block cache or not. */ public static final boolean DEFAULT_BLOCK_CACHE_ENABLED = false; - + /** * Default maximum length of cell contents. */ @@ -96,56 +92,71 @@ public class HColumnDescriptor implements WritableComparable { */ public static final BloomFilterDescriptor DEFAULT_BLOOM_FILTER_DESCRIPTOR = null; - + // Column family name - private Text name; + private byte [] name; // Number of versions to keep - private int maxVersions; + private int maxVersions = DEFAULT_N_VERSIONS; // Compression setting if any - private CompressionType compressionType; + private CompressionType compressionType = DEFAULT_COMPRESSION_TYPE; // Serve reads from in-memory cache - private boolean inMemory; + private boolean inMemory = DEFAULT_IN_MEMORY; // Serve reads from in-memory block cache - private boolean blockCacheEnabled; + private boolean blockCacheEnabled = DEFAULT_BLOCK_CACHE_ENABLED; // Maximum value size - private int maxValueLength; + private int maxValueLength = Integer.MAX_VALUE; // Time to live of cell contents, in seconds from last timestamp - private int timeToLive; + private int timeToLive = HConstants.FOREVER; // True if bloom filter was specified - private boolean bloomFilterSpecified; + private boolean bloomFilterSpecified = false; // Descriptor of bloom filter - private BloomFilterDescriptor bloomFilter; - // Version number of this class - private byte versionNumber; - // Family name without the ':' - private transient Text familyName = null; - + private BloomFilterDescriptor bloomFilter = DEFAULT_BLOOM_FILTER_DESCRIPTOR; + /** * Default constructor. Must be present for Writable. */ public HColumnDescriptor() { - this(null); + this.name = null; } - + /** * Construct a column descriptor specifying only the family name * The other attributes are defaulted. * * @param columnName - column family name */ - public HColumnDescriptor(String columnName) { - this(columnName == null || columnName.length() <= 0? - new Text(): new Text(columnName), + public HColumnDescriptor(final String columnName) { + this(Bytes.toBytes(columnName)); + } + + /** + * Construct a column descriptor specifying only the family name + * The other attributes are defaulted. + * + * @param columnName - column family name + */ + public HColumnDescriptor(final Text columnName) { + this(columnName.getBytes()); + } + + /** + * Construct a column descriptor specifying only the family name + * The other attributes are defaulted. + * + * @param columnName Column family name. Must have the ':' ending. + */ + public HColumnDescriptor(final byte [] columnName) { + this (columnName == null || columnName.length <= 0? + HConstants.EMPTY_BYTE_ARRAY: columnName, DEFAULT_N_VERSIONS, DEFAULT_COMPRESSION_TYPE, DEFAULT_IN_MEMORY, DEFAULT_BLOCK_CACHE_ENABLED, Integer.MAX_VALUE, DEFAULT_TIME_TO_LIVE, DEFAULT_BLOOM_FILTER_DESCRIPTOR); } - + /** * Constructor - * Specify all parameters. - * @param name Column family name + * @param columnName Column family name. Must have the ':' ending. * @param maxVersions Maximum number of versions to keep * @param compression Compression type * @param inMemory If true, column data should be kept in an HRegionServer's @@ -161,25 +172,14 @@ public class HColumnDescriptor implements WritableComparable { * end in a : * @throws IllegalArgumentException if the number of versions is <= 0 */ - public HColumnDescriptor(final Text name, final int maxVersions, + public HColumnDescriptor(final byte [] columnName, final int maxVersions, final CompressionType compression, final boolean inMemory, final boolean blockCacheEnabled, final int maxValueLength, final int timeToLive, final BloomFilterDescriptor bloomFilter) { - String familyStr = name.toString(); - // Test name if not null (It can be null when deserializing after - // construction but before we've read in the fields); - if (familyStr.length() > 0) { - Matcher m = LEGAL_FAMILY_NAME.matcher(familyStr); - if(m == null || !m.matches()) { - throw new IllegalArgumentException("Illegal family name <" + name + - ">. Family names can only contain " + - "'word characters' and must end with a ':'"); - } - } - this.name = name; - - if(maxVersions <= 0) { + isLegalFamilyName(columnName); + this.name = stripColon(columnName); + if (maxVersions <= 0) { // TODO: Allow maxVersion of 0 to be the way you say "Keep all versions". // Until there is support, consider 0 or < 0 -- a configuration error. throw new IllegalArgumentException("Maximum versions must be positive"); @@ -191,26 +191,49 @@ public class HColumnDescriptor implements WritableComparable { this.timeToLive = timeToLive; this.bloomFilter = bloomFilter; this.bloomFilterSpecified = this.bloomFilter == null ? false : true; - this.versionNumber = COLUMN_DESCRIPTOR_VERSION; this.compressionType = compression; } + + private static byte [] stripColon(final byte [] n) { + byte [] result = new byte [n.length - 1]; + // Have the stored family name be absent the colon delimiter + System.arraycopy(n, 0, result, 0, n.length - 1); + return result; + } + + /** + * @param b Family name. + * @return b + * @throws IllegalArgumentException If not null and not a legitimate family + * name: i.e. 'printable' and ends in a ':' (Null passes are allowed because + * b can be null when deserializing). + */ + public static byte [] isLegalFamilyName(final byte [] b) { + if (b == null) { + return b; + } + if (b[b.length - 1] != ':') { + throw new IllegalArgumentException("Family names must end in a colon: " + + Bytes.toString(b)); + } + for (int i = 0; i < (b.length - 1); i++) { + if (Character.isLetterOrDigit(b[i]) || b[i] == '_' || b[i] == '.') { + continue; + } + throw new IllegalArgumentException("Illegal character <" + b[i] + + ">. Family names can only contain 'word characters' and must end" + + "with a colon: " + Bytes.toString(b)); + } + return b; + } - /** @return name of column family */ - public Text getName() { + /** + * @return Name of this column family + */ + public byte [] getName() { return name; } - /** @return name of column family without trailing ':' */ - public synchronized Text getFamilyName() { - if (name != null) { - if (familyName == null) { - familyName = new TextSequence(name, 0, name.getLength() - 1).toText(); - } - return familyName; - } - return null; - } - /** @return compression type being used for the column family */ public CompressionType getCompression() { return this.compressionType; @@ -266,9 +289,7 @@ public class HColumnDescriptor implements WritableComparable { /** {@inheritDoc} */ @Override public String toString() { - // Output a name minus ':'. - String tmp = name.toString(); - return "{name: " + tmp.substring(0, tmp.length() - 1) + + return "{name: " + Bytes.toString(name) + ", max versions: " + maxVersions + ", compression: " + this.compressionType + ", in memory: " + inMemory + ", block cache enabled: " + blockCacheEnabled + @@ -290,7 +311,7 @@ public class HColumnDescriptor implements WritableComparable { /** {@inheritDoc} */ @Override public int hashCode() { - int result = this.name.hashCode(); + int result = Bytes.hashCode(this.name); result ^= Integer.valueOf(this.maxVersions).hashCode(); result ^= this.compressionType.hashCode(); result ^= Boolean.valueOf(this.inMemory).hashCode(); @@ -298,8 +319,8 @@ public class HColumnDescriptor implements WritableComparable { result ^= Integer.valueOf(this.maxValueLength).hashCode(); result ^= Integer.valueOf(this.timeToLive).hashCode(); result ^= Boolean.valueOf(this.bloomFilterSpecified).hashCode(); - result ^= Byte.valueOf(this.versionNumber).hashCode(); - if(this.bloomFilterSpecified) { + result ^= Byte.valueOf(COLUMN_DESCRIPTOR_VERSION).hashCode(); + if (this.bloomFilterSpecified) { result ^= this.bloomFilter.hashCode(); } return result; @@ -309,8 +330,14 @@ public class HColumnDescriptor implements WritableComparable { /** {@inheritDoc} */ public void readFields(DataInput in) throws IOException { - this.versionNumber = in.readByte(); - this.name.readFields(in); + int versionNumber = in.readByte(); + if (versionNumber <= 2) { + Text t = new Text(); + t.readFields(in); + this.name = t.getBytes(); + } else { + this.name = Bytes.readByteArray(in); + } this.maxVersions = in.readInt(); int ordinal = in.readInt(); this.compressionType = CompressionType.values()[ordinal]; @@ -323,19 +350,19 @@ public class HColumnDescriptor implements WritableComparable { bloomFilter.readFields(in); } - if (this.versionNumber > 1) { + if (versionNumber > 1) { this.blockCacheEnabled = in.readBoolean(); } - if (this.versionNumber > 2) { + if (versionNumber > 2) { this.timeToLive = in.readInt(); } } /** {@inheritDoc} */ public void write(DataOutput out) throws IOException { - out.writeByte(this.versionNumber); - this.name.write(out); + out.writeByte(COLUMN_DESCRIPTOR_VERSION); + Bytes.writeByteArray(out, this.name); out.writeInt(this.maxVersions); out.writeInt(this.compressionType.ordinal()); out.writeBoolean(this.inMemory); @@ -345,28 +372,16 @@ public class HColumnDescriptor implements WritableComparable { if(bloomFilterSpecified) { bloomFilter.write(out); } - - if (this.versionNumber > 1) { - out.writeBoolean(this.blockCacheEnabled); - } - - if (this.versionNumber > 2) { - out.writeInt(this.timeToLive); - } + out.writeBoolean(this.blockCacheEnabled); + out.writeInt(this.timeToLive); } // Comparable /** {@inheritDoc} */ public int compareTo(Object o) { - // NOTE: we don't do anything with the version number yet. - // Version numbers will come into play when we introduce an incompatible - // change in the future such as the addition of access control lists. - HColumnDescriptor other = (HColumnDescriptor)o; - - int result = this.name.compareTo(other.getName()); - + int result = Bytes.compareTo(this.name, other.getName()); if(result == 0) { result = Integer.valueOf(this.maxVersions).compareTo( Integer.valueOf(other.maxVersions)); @@ -426,4 +441,4 @@ public class HColumnDescriptor implements WritableComparable { return result; } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/HConstants.java b/src/java/org/apache/hadoop/hbase/HConstants.java index fef78c53bcd..71c9072d860 100644 --- a/src/java/org/apache/hadoop/hbase/HConstants.java +++ b/src/java/org/apache/hadoop/hbase/HConstants.java @@ -19,8 +19,8 @@ */ package org.apache.hadoop.hbase; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.util.Bytes; /** * HConstants holds a bunch of HBase-related constants @@ -30,6 +30,8 @@ public interface HConstants { /** long constant for zero */ static final Long ZERO_L = Long.valueOf(0L); + static final String NINES = "99999999999999"; + // For migration /** name of version file */ @@ -117,62 +119,69 @@ public interface HConstants { // should go down. /** The root table's name.*/ - static final Text ROOT_TABLE_NAME = new Text("-ROOT-"); + static final byte [] ROOT_TABLE_NAME = Bytes.toBytes("-ROOT-"); /** The META table's name. */ - static final Text META_TABLE_NAME = new Text(".META."); + static final byte [] META_TABLE_NAME = Bytes.toBytes(".META."); // Defines for the column names used in both ROOT and META HBase 'meta' tables. /** The ROOT and META column family (string) */ static final String COLUMN_FAMILY_STR = "info:"; - /** The ROOT and META column family (Text) */ - static final Text COLUMN_FAMILY = new Text(COLUMN_FAMILY_STR); + /** The ROOT and META column family */ + static final byte [] COLUMN_FAMILY = Bytes.toBytes(COLUMN_FAMILY_STR); /** Array of meta column names */ - static final Text [] COLUMN_FAMILY_ARRAY = new Text [] {COLUMN_FAMILY}; + static final byte[][] COLUMN_FAMILY_ARRAY = new byte[][] {COLUMN_FAMILY}; /** ROOT/META column family member - contains HRegionInfo */ - static final Text COL_REGIONINFO = new Text(COLUMN_FAMILY + "regioninfo"); + static final byte [] COL_REGIONINFO = + Bytes.toBytes(COLUMN_FAMILY_STR + "regioninfo"); /** Array of column - contains HRegionInfo */ - static final Text[] COL_REGIONINFO_ARRAY = new Text [] {COL_REGIONINFO}; + static final byte[][] COL_REGIONINFO_ARRAY = new byte[][] {COL_REGIONINFO}; /** ROOT/META column family member - contains HServerAddress.toString() */ - static final Text COL_SERVER = new Text(COLUMN_FAMILY + "server"); + static final byte[] COL_SERVER = Bytes.toBytes(COLUMN_FAMILY_STR + "server"); /** ROOT/META column family member - contains server start code (a long) */ - static final Text COL_STARTCODE = new Text(COLUMN_FAMILY + "serverstartcode"); + static final byte [] COL_STARTCODE = + Bytes.toBytes(COLUMN_FAMILY_STR + "serverstartcode"); /** the lower half of a split region */ - static final Text COL_SPLITA = new Text(COLUMN_FAMILY_STR + "splitA"); + static final byte [] COL_SPLITA = Bytes.toBytes(COLUMN_FAMILY_STR + "splitA"); /** the upper half of a split region */ - static final Text COL_SPLITB = new Text(COLUMN_FAMILY_STR + "splitB"); + static final byte [] COL_SPLITB = Bytes.toBytes(COLUMN_FAMILY_STR + "splitB"); /** All the columns in the catalog -ROOT- and .META. tables. */ - static final Text[] ALL_META_COLUMNS = {COL_REGIONINFO, COL_SERVER, + static final byte[][] ALL_META_COLUMNS = {COL_REGIONINFO, COL_SERVER, COL_STARTCODE, COL_SPLITA, COL_SPLITB}; // Other constants /** - * An empty instance of Text. + * An empty instance. */ - static final Text EMPTY_TEXT = new Text(); + static final byte [] EMPTY_BYTE_ARRAY = new byte [0]; /** * Used by scanners, etc when they want to start at the beginning of a region */ - static final Text EMPTY_START_ROW = EMPTY_TEXT; + static final byte [] EMPTY_START_ROW = EMPTY_BYTE_ARRAY; + + /** + * Last row in a table. + */ + static final byte [] EMPTY_END_ROW = EMPTY_START_ROW; /** * Used by scanners and others when they're trying to detect the end of a * table */ - static final Text LAST_ROW = EMPTY_TEXT; + static final byte [] LAST_ROW = EMPTY_BYTE_ARRAY; /** When we encode strings, we always specify UTF8 encoding */ static final String UTF8_ENCODING = "UTF-8"; diff --git a/src/java/org/apache/hadoop/hbase/HMerge.java b/src/java/org/apache/hadoop/hbase/HMerge.java index a9c0c35ffb6..354c620ecf2 100644 --- a/src/java/org/apache/hadoop/hbase/HMerge.java +++ b/src/java/org/apache/hadoop/hbase/HMerge.java @@ -22,28 +22,25 @@ package org.apache.hadoop.hbase; import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.Map; import java.util.NoSuchElementException; import java.util.Random; -import java.util.SortedMap; import java.util.TreeMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Text; - import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Scanner; import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.regionserver.HLog; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; /** @@ -57,7 +54,9 @@ class HMerge implements HConstants { /* * Not instantiable */ - private HMerge() {} + private HMerge() { + super(); + } /** * Scans the table and merges two adjacent regions if they are small. This @@ -73,13 +72,13 @@ class HMerge implements HConstants { * @throws IOException */ public static void merge(HBaseConfiguration conf, FileSystem fs, - Text tableName) + final byte [] tableName) throws IOException { HConnection connection = HConnectionManager.getConnection(conf); boolean masterIsRunning = connection.isMasterRunning(); HConnectionManager.deleteConnection(conf); - if(tableName.equals(META_TABLE_NAME)) { - if(masterIsRunning) { + if (Bytes.equals(tableName, META_TABLE_NAME)) { + if (masterIsRunning) { throw new IllegalStateException( "Can not compact META table if instance is on-line"); } @@ -101,9 +100,9 @@ class HMerge implements HConstants { private final long maxFilesize; - protected Merger(HBaseConfiguration conf, FileSystem fs, Text tableName) - throws IOException { - + protected Merger(HBaseConfiguration conf, FileSystem fs, + final byte [] tableName) + throws IOException { this.conf = conf; this.fs = fs; this.maxFilesize = @@ -184,19 +183,21 @@ class HMerge implements HConstants { protected abstract HRegionInfo[] next() throws IOException; - protected abstract void updateMeta(Text oldRegion1, Text oldRegion2, - HRegion newRegion) throws IOException; + protected abstract void updateMeta(final byte [] oldRegion1, + final byte [] oldRegion2, HRegion newRegion) + throws IOException; } /** Instantiated to compact a normal user table */ private static class OnlineMerger extends Merger { - private final Text tableName; + private final byte [] tableName; private final HTable table; private final Scanner metaScanner; private HRegionInfo latestRegion; - OnlineMerger(HBaseConfiguration conf, FileSystem fs, Text tableName) + OnlineMerger(HBaseConfiguration conf, FileSystem fs, + final byte [] tableName) throws IOException { super(conf, fs, tableName); this.tableName = tableName; @@ -217,7 +218,7 @@ class HMerge implements HConstants { COL_REGIONINFO); } HRegionInfo region = Writables.getHRegionInfo(regionInfo.getValue()); - if (!region.getTableDesc().getName().equals(this.tableName)) { + if (!Bytes.equals(region.getTableDesc().getName(), this.tableName)) { return null; } checkOfflined(region); @@ -276,16 +277,16 @@ class HMerge implements HConstants { } @Override - protected void updateMeta(Text oldRegion1, Text oldRegion2, + protected void updateMeta(final byte [] oldRegion1, + final byte [] oldRegion2, HRegion newRegion) throws IOException { - Text[] regionsToDelete = {oldRegion1, oldRegion2}; - for(int r = 0; r < regionsToDelete.length; r++) { - if(regionsToDelete[r].equals(latestRegion.getRegionName())) { + byte[][] regionsToDelete = {oldRegion1, oldRegion2}; + for (int r = 0; r < regionsToDelete.length; r++) { + if(Bytes.equals(regionsToDelete[r], latestRegion.getRegionName())) { latestRegion = null; } table.deleteAll(regionsToDelete[r]); - if(LOG.isDebugEnabled()) { LOG.debug("updated columns in row: " + regionsToDelete[r]); } @@ -321,15 +322,16 @@ class HMerge implements HConstants { // Scan root region to find all the meta regions root = new HRegion(rootTableDir, hlog, fs, conf, - HRegionInfo.rootRegionInfo, null, null); + HRegionInfo.ROOT_REGIONINFO, null, null); InternalScanner rootScanner = - root.getScanner(COL_REGIONINFO_ARRAY, new Text(), + root.getScanner(COL_REGIONINFO_ARRAY, HConstants.EMPTY_START_ROW, HConstants.LATEST_TIMESTAMP, null); try { HStoreKey key = new HStoreKey(); - TreeMap results = new TreeMap(); + TreeMap results = + new TreeMap(Bytes.BYTES_COMPARATOR); while(rootScanner.next(key, results)) { for(byte [] b: results.values()) { HRegionInfo info = Writables.getHRegionInfoOrNull(b); @@ -360,13 +362,10 @@ class HMerge implements HConstants { } @Override - protected void updateMeta(Text oldRegion1, Text oldRegion2, - HRegion newRegion) throws IOException { - - Text[] regionsToDelete = { - oldRegion1, - oldRegion2 - }; + protected void updateMeta(final byte [] oldRegion1, + final byte [] oldRegion2, HRegion newRegion) + throws IOException { + byte[][] regionsToDelete = {oldRegion1, oldRegion2}; for(int r = 0; r < regionsToDelete.length; r++) { BatchUpdate b = new BatchUpdate(regionsToDelete[r]); b.delete(COL_REGIONINFO); diff --git a/src/java/org/apache/hadoop/hbase/HMsg.java b/src/java/org/apache/hadoop/hbase/HMsg.java index 8959a4db3ce..572540e515d 100644 --- a/src/java/org/apache/hadoop/hbase/HMsg.java +++ b/src/java/org/apache/hadoop/hbase/HMsg.java @@ -188,7 +188,7 @@ public class HMsg implements Writable { message.append(") : "); break; } - message.append(info == null ? "null" : info.getRegionName()); + message.append(info == null ? "null": info.getRegionNameAsString()); return message.toString(); } @@ -211,4 +211,4 @@ public class HMsg implements Writable { this.msg = in.readByte(); this.info.readFields(in); } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/HRegionInfo.java b/src/java/org/apache/hadoop/hbase/HRegionInfo.java index 40ca71f561c..91f96b03696 100644 --- a/src/java/org/apache/hadoop/hbase/HRegionInfo.java +++ b/src/java/org/apache/hadoop/hbase/HRegionInfo.java @@ -23,10 +23,9 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.WritableComparable; - +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JenkinsHash; +import org.apache.hadoop.io.WritableComparable; /** * HRegion information. @@ -38,21 +37,20 @@ public class HRegionInfo implements WritableComparable { * @param regionName * @return the encodedName */ - public static String encodeRegionName(final Text regionName) { - return String.valueOf(Math.abs( - JenkinsHash.hash(regionName.getBytes(), regionName.getLength(), 0))); + public static int encodeRegionName(final byte [] regionName) { + return Math.abs(JenkinsHash.hash(regionName, regionName.length, 0)); } /** delimiter used between portions of a region name */ - private static final String DELIMITER = ","; + public static final int DELIMITER = ','; /** HRegionInfo for root region */ - public static final HRegionInfo rootRegionInfo = - new HRegionInfo(0L, HTableDescriptor.rootTableDesc); + public static final HRegionInfo ROOT_REGIONINFO = + new HRegionInfo(0L, HTableDescriptor.ROOT_TABLEDESC); /** HRegionInfo for first meta region */ - public static final HRegionInfo firstMetaRegionInfo = - new HRegionInfo(1L, HTableDescriptor.metaTableDesc); + public static final HRegionInfo FIRST_META_REGIONINFO = + new HRegionInfo(1L, HTableDescriptor.META_TABLEDESC); /** * Extracts table name prefix from a region name. @@ -60,26 +58,34 @@ public class HRegionInfo implements WritableComparable { * @param regionName A region name. * @return The table prefix of a region name. */ - public static Text getTableNameFromRegionName(final Text regionName) { - int offset = regionName.find(DELIMITER); + public static byte [] getTableNameFromRegionName(final byte [] regionName) { + int offset = -1; + for (int i = 0; i < regionName.length; i++) { + if (regionName[i] == DELIMITER) { + offset = i; + break; + } + } if (offset == -1) { - throw new IllegalArgumentException(regionName.toString() + " does not " + - "contain '" + DELIMITER + "' character"); + throw new IllegalArgumentException(Bytes.toString(regionName) + + " does not contain '" + DELIMITER + "' character"); } byte [] tableName = new byte[offset]; - System.arraycopy(regionName.getBytes(), 0, tableName, 0, offset); - return new Text(tableName); + System.arraycopy(regionName, 0, tableName, 0, offset); + return tableName; } - private Text endKey; - private boolean offLine; - private long regionId; - private Text regionName; - private boolean split; - private Text startKey; - private HTableDescriptor tableDesc; - private int hashCode; - private transient String encodedName = null; + private byte [] endKey = HConstants.EMPTY_BYTE_ARRAY; + private boolean offLine = false; + private long regionId = -1; + private byte [] regionName = HConstants.EMPTY_BYTE_ARRAY; + private String regionNameStr = ""; + private boolean split = false; + private byte [] startKey = HConstants.EMPTY_BYTE_ARRAY; + private HTableDescriptor tableDesc = null; + private int hashCode = -1; + public static final int NO_HASH = -1; + private volatile int encodedName = NO_HASH; private void setHashCode() { int result = this.regionName.hashCode(); @@ -91,29 +97,21 @@ public class HRegionInfo implements WritableComparable { this.hashCode = result; } - /** Used to construct the HRegionInfo for the root and first meta regions */ + /** + * Private constructor used constructing HRegionInfo for the catalog 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(); + this.regionName = createRegionName(tableDesc.getName(), null, regionId); + this.regionNameStr = Bytes.toString(this.regionName); setHashCode(); } /** Default constructor - creates empty object */ public HRegionInfo() { - this.endKey = new Text(); - this.offLine = false; - this.regionId = 0; - this.regionName = new Text(); - this.split = false; - this.startKey = new Text(); this.tableDesc = new HTableDescriptor(); - this.hashCode = 0; } /** @@ -124,8 +122,9 @@ public class HRegionInfo implements WritableComparable { * @param endKey end of key range * @throws IllegalArgumentException */ - public HRegionInfo(HTableDescriptor tableDesc, Text startKey, Text endKey) - throws IllegalArgumentException { + public HRegionInfo(final HTableDescriptor tableDesc, final byte [] startKey, + final byte [] endKey) + throws IllegalArgumentException { this(tableDesc, startKey, endKey, false); } @@ -139,38 +138,65 @@ public class HRegionInfo implements WritableComparable { * regions that may or may not hold references to this region. * @throws IllegalArgumentException */ - public HRegionInfo(HTableDescriptor tableDesc, Text startKey, Text endKey, - final boolean split) throws IllegalArgumentException { - - if(tableDesc == null) { + public HRegionInfo(HTableDescriptor tableDesc, final byte [] startKey, + final byte [] endKey, final boolean split) + throws IllegalArgumentException { + if (tableDesc == null) { throw new IllegalArgumentException("tableDesc cannot be null"); } - - this.endKey = new Text(); - if(endKey != null) { - this.endKey.set(endKey); - } - this.offLine = false; this.regionId = System.currentTimeMillis(); - - this.regionName = new Text(tableDesc.getName().toString() + DELIMITER + - (startKey == null ? "" : startKey.toString()) + DELIMITER + - regionId); - + this.regionName = createRegionName(tableDesc.getName(), startKey, regionId); + this.regionNameStr = Bytes.toString(this.regionName); this.split = split; - - this.startKey = new Text(); - if(startKey != null) { - this.startKey.set(startKey); - } - + this.endKey = endKey == null? HConstants.EMPTY_END_ROW: endKey.clone(); + this.startKey = startKey == null? + HConstants.EMPTY_START_ROW: startKey.clone(); this.tableDesc = tableDesc; setHashCode(); } + private static byte [] createRegionName(final byte [] tableName, + final byte [] startKey, final long regionid) { + return createRegionName(tableName, startKey, Long.toString(regionid)); + } + + /** + * Make a region name of passed parameters. + * @param tableName + * @param startKey Can be null + * @param id Region id. + * @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)); + } + /** + * Make a region name of passed parameters. + * @param tableName + * @param startKey Can be null + * @param id Region id + * @return Region name made of passed tableName, startKey and id + */ + public static byte [] createRegionName(final byte [] tableName, + final byte [] startKey, final byte [] id) { + byte [] b = new byte [tableName.length + 2 + id.length + + (startKey == null? 0: startKey.length)]; + int offset = tableName.length; + System.arraycopy(tableName, 0, b, 0, offset); + b[offset++] = DELIMITER; + if (startKey != null && startKey.length > 0) { + System.arraycopy(startKey, 0, b, offset, startKey.length); + offset += startKey.length; + } + b[offset++] = DELIMITER; + System.arraycopy(id, 0, b, offset, id.length); + return b; + } + /** @return the endKey */ - public Text getEndKey(){ + public byte [] getEndKey(){ return endKey; } @@ -179,21 +205,31 @@ public class HRegionInfo implements WritableComparable { return regionId; } - /** @return the regionName */ - public Text getRegionName(){ + /** + * @return the regionName as an array of bytes. + * @see #getRegionNameAsString() + */ + public byte [] getRegionName(){ return regionName; } + + /** + * @return Region name as a String for use in logging, etc. + */ + public String getRegionNameAsString() { + return this.regionNameStr; + } /** @return the encoded region name */ - public synchronized String getEncodedName() { - if (encodedName == null) { - encodedName = encodeRegionName(regionName); + public synchronized int getEncodedName() { + if (this.encodedName == NO_HASH) { + this.encodedName = encodeRegionName(this.regionName); } - return encodedName; + return this.encodedName; } /** @return the startKey */ - public Text getStartKey(){ + public byte [] getStartKey(){ return startKey; } @@ -250,8 +286,9 @@ public class HRegionInfo implements WritableComparable { */ @Override public String toString() { - return "regionname: " + this.regionName.toString() + ", startKey: <" + - this.startKey.toString() + ">, endKey: <" + this.endKey.toString() + + return "regionname: " + this.regionNameStr + ", startKey: <" + + Bytes.toString(this.startKey) + ">, endKey: <" + + Bytes.toString(this.endKey) + ">, encodedName: " + getEncodedName() + "," + (isOffline()? " offline: true,": "") + (isSplit()? " split: true,": "") + " tableDesc: {" + this.tableDesc.toString() + "}"; @@ -281,12 +318,12 @@ public class HRegionInfo implements WritableComparable { * {@inheritDoc} */ public void write(DataOutput out) throws IOException { - endKey.write(out); + Bytes.writeByteArray(out, endKey); out.writeBoolean(offLine); out.writeLong(regionId); - regionName.write(out); + Bytes.writeByteArray(out, regionName); out.writeBoolean(split); - startKey.write(out); + Bytes.writeByteArray(out, startKey); tableDesc.write(out); out.writeInt(hashCode); } @@ -295,12 +332,13 @@ public class HRegionInfo implements WritableComparable { * {@inheritDoc} */ public void readFields(DataInput in) throws IOException { - this.endKey.readFields(in); + this.endKey = Bytes.readByteArray(in); this.offLine = in.readBoolean(); this.regionId = in.readLong(); - this.regionName.readFields(in); + this.regionName = Bytes.readByteArray(in); + this.regionNameStr = Bytes.toString(this.regionName); this.split = in.readBoolean(); - this.startKey.readFields(in); + this.startKey = Bytes.readByteArray(in); this.tableDesc.readFields(in); this.hashCode = in.readInt(); } @@ -322,12 +360,12 @@ public class HRegionInfo implements WritableComparable { } // Compare start keys. - result = this.startKey.compareTo(other.startKey); + result = Bytes.compareTo(this.startKey, other.startKey); if (result != 0) { return result; } // Compare end keys. - return this.endKey.compareTo(other.endKey); + return Bytes.compareTo(this.endKey, other.endKey); } } diff --git a/src/java/org/apache/hadoop/hbase/HStoreKey.java b/src/java/org/apache/hadoop/hbase/HStoreKey.java index dbc740dcafa..3c57301d8bc 100644 --- a/src/java/org/apache/hadoop/hbase/HStoreKey.java +++ b/src/java/org/apache/hadoop/hbase/HStoreKey.java @@ -19,14 +19,16 @@ */ package org.apache.hadoop.hbase; -import org.apache.hadoop.hbase.io.TextSequence; -import org.apache.hadoop.io.*; -import java.io.*; -import java.nio.ByteBuffer; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.WritableComparable; /** - * A Key for a stored row + * A Key for a stored row. */ public class HStoreKey implements WritableComparable { /** @@ -34,14 +36,13 @@ public class HStoreKey implements WritableComparable { */ public static final char COLUMN_FAMILY_DELIMITER = ':'; - private Text row; - private Text column; - private long timestamp; - + private byte [] row = HConstants.EMPTY_BYTE_ARRAY; + private byte [] column = HConstants.EMPTY_BYTE_ARRAY; + private long timestamp = Long.MAX_VALUE; /** Default constructor used in conjunction with Writable interface */ public HStoreKey() { - this(new Text()); + super(); } /** @@ -51,10 +52,21 @@ public class HStoreKey implements WritableComparable { * * @param row - row key */ - public HStoreKey(Text row) { + public HStoreKey(final byte [] row) { this(row, Long.MAX_VALUE); } - + + /** + * Create an HStoreKey specifying only the row + * The column defaults to the empty string and the time stamp defaults to + * Long.MAX_VALUE + * + * @param row - row key + */ + public HStoreKey(final String row) { + this(row, Long.MAX_VALUE); + } + /** * Create an HStoreKey specifying the row and timestamp * The column name defaults to the empty string @@ -62,10 +74,21 @@ public class HStoreKey implements WritableComparable { * @param row row key * @param timestamp timestamp value */ - public HStoreKey(Text row, long timestamp) { - this(row, new Text(), timestamp); + public HStoreKey(final byte [] row, long timestamp) { + this(row, HConstants.EMPTY_BYTE_ARRAY, timestamp); } - + + /** + * Create an HStoreKey specifying the row and timestamp + * The column name defaults to the empty string + * + * @param row row key + * @param timestamp timestamp value + */ + public HStoreKey(final String row, long timestamp) { + this (row, "", timestamp); + } + /** * Create an HStoreKey specifying the row and column names * The timestamp defaults to LATEST_TIMESTAMP @@ -73,28 +96,51 @@ public class HStoreKey implements WritableComparable { * @param row row key * @param column column key */ - public HStoreKey(Text row, Text column) { + public HStoreKey(final String row, final String column) { this(row, column, HConstants.LATEST_TIMESTAMP); } - + + /** + * Create an HStoreKey specifying the row and column names + * The timestamp defaults to LATEST_TIMESTAMP + * + * @param row row key + * @param column column key + */ + public HStoreKey(final byte [] row, final byte [] column) { + this(row, column, HConstants.LATEST_TIMESTAMP); + } + /** * Create an HStoreKey specifying all the fields - * + * Does not make copies of the passed byte arrays. Presumes the passed + * arrays immutable. * @param row row key * @param column column key * @param timestamp timestamp value */ - public HStoreKey(Text row, Text column, long timestamp) { - // Make copies by doing 'new Text(arg)'. - this.row = new Text(row); - this.column = new Text(column); + public HStoreKey(final String row, final String column, long timestamp) { + this (Bytes.toBytes(row), Bytes.toBytes(column), timestamp); + } + + /** + * Create an HStoreKey specifying all the fields + * Does not make copies of the passed byte arrays. Presumes the passed + * arrays immutable. + * @param row row key + * @param column column key + * @param timestamp timestamp value + */ + public HStoreKey(final byte [] row, final byte [] column, long timestamp) { + // Make copies + this.row = row; + this.column = column; this.timestamp = timestamp; } /** @return Approximate size in bytes of this key. */ public long getSize() { - return this.row.getLength() + this.column.getLength() + - 8 /* There is no sizeof in java. Presume long is 8 (64bit machine)*/; + return this.row.length + this.column.length + Bytes.SIZEOF_LONG; } /** @@ -111,19 +157,19 @@ public class HStoreKey implements WritableComparable { * * @param newrow new row key value */ - public void setRow(Text newrow) { - this.row.set(newrow); + public void setRow(byte [] newrow) { + this.row = newrow; } /** - * Change the value of the column key + * Change the value of the column in this key * - * @param newcol new column key value + * @param c new column family value */ - public void setColumn(Text newcol) { - this.column.set(newcol); + public void setColumn(byte [] c) { + this.column = c; } - + /** * Change the value of the timestamp field * @@ -145,18 +191,18 @@ public class HStoreKey implements WritableComparable { } /** @return value of row key */ - public Text getRow() { + public byte [] getRow() { return row; } - /** @return value of column key */ - public Text getColumn() { - return column; + /** @return value of column */ + public byte [] getColumn() { + return this.column; } - + /** @return value of timestamp */ public long getTimestamp() { - return timestamp; + return this.timestamp; } /** @@ -167,8 +213,8 @@ public class HStoreKey implements WritableComparable { * @see #matchesRowFamily(HStoreKey) */ public boolean matchesRowCol(HStoreKey other) { - return this.row.compareTo(other.row) == 0 - && this.column.compareTo(other.column) == 0; + return Bytes.equals(this.row, other.row) && + Bytes.equals(column, other.column); } /** @@ -181,8 +227,8 @@ public class HStoreKey implements WritableComparable { * @see #matchesRowFamily(HStoreKey) */ public boolean matchesWithoutColumn(HStoreKey other) { - return this.row.compareTo(other.row) == 0 - && this.timestamp >= other.getTimestamp(); + return Bytes.equals(this.row, other.row) && + this.timestamp >= other.getTimestamp(); } /** @@ -191,21 +237,21 @@ public class HStoreKey implements WritableComparable { * @param that Key to compare against. Compares row and column family * * @return true if same row and column family - * @throws InvalidColumnNameException * @see #matchesRowCol(HStoreKey) * @see #matchesWithoutColumn(HStoreKey) */ - public boolean matchesRowFamily(HStoreKey that) - throws InvalidColumnNameException { - return this.row.compareTo(that.row) == 0 && - extractFamily(this.column). - compareTo(extractFamily(that.getColumn())) == 0; + public boolean matchesRowFamily(HStoreKey that) { + int delimiterIndex = getFamilyDelimiterIndex(this.column); + return Bytes.equals(this.row, that.row) && + Bytes.compareTo(this.column, 0, delimiterIndex, that.column, 0, + delimiterIndex) == 0; } /** {@inheritDoc} */ @Override public String toString() { - return row.toString() + "/" + column.toString() + "/" + timestamp; + return Bytes.toString(this.row) + "/" + Bytes.toString(this.column) + "/" + + timestamp; } /** {@inheritDoc} */ @@ -228,11 +274,14 @@ public class HStoreKey implements WritableComparable { /** {@inheritDoc} */ public int compareTo(Object o) { HStoreKey other = (HStoreKey)o; - int result = this.row.compareTo(other.row); + int result = Bytes.compareTo(this.row, other.row); if (result != 0) { return result; } - result = this.column.compareTo(other.column); + result = this.column == null && other.column == null? 0: + this.column == null && other.column != null? -1: + this.column != null && other.column == null? 1: + Bytes.compareTo(this.column, other.column); if (result != 0) { return result; } @@ -248,108 +297,136 @@ public class HStoreKey implements WritableComparable { return result; } - // Writable - - /** {@inheritDoc} */ - public void write(DataOutput out) throws IOException { - row.write(out); - column.write(out); - out.writeLong(timestamp); + /** + * @param column + * @return New byte array that holds column family prefix. + * @see #parseColumn(byte[]) + */ + public static byte [] getFamily(final byte [] column) { + int index = getFamilyDelimiterIndex(column); + if (index <= 0) { + throw new IllegalArgumentException("No ':' delimiter between " + + "column family and qualifier in the passed column name <" + + Bytes.toString(column) + ">"); + } + byte [] result = new byte[index]; + System.arraycopy(column, 0, result, 0, index); + return result; + } + + /** + * @param column + * @return Return hash of family portion of passed column. + */ + public static Integer getFamilyMapKey(final byte [] column) { + int index = getFamilyDelimiterIndex(column); + // If index < -1, presume passed column is a family name absent colon + // delimiter + return Bytes.mapKey(column, index > 0? index: column.length); + } + + /** + * @param family + * @param column + * @return True if column has a family of family. + */ + public static boolean matchingFamily(final byte [] family, + final byte [] column) { + // Make sure index of the ':' is at same offset. + int index = getFamilyDelimiterIndex(column); + if (index != family.length) { + return false; + } + return Bytes.compareTo(family, 0, index, column, 0, index) == 0; + } + + /** + * @param family + * @return Return family plus the family delimiter. + */ + public static byte [] addDelimiter(final byte [] family) { + // Manufacture key by adding delimiter to the passed in colFamily. + byte [] familyPlusDelimiter = new byte [family.length + 1]; + System.arraycopy(family, 0, familyPlusDelimiter, 0, family.length); + familyPlusDelimiter[family.length] = HStoreKey.COLUMN_FAMILY_DELIMITER; + return familyPlusDelimiter; } - /** {@inheritDoc} */ - public void readFields(DataInput in) throws IOException { - row.readFields(in); - column.readFields(in); - timestamp = in.readLong(); - } - - // Statics - // TODO: Move these utility methods elsewhere (To a Column class?). - /** - * Extracts the column family name from a column - * For example, returns 'info' if the specified column was 'info:server' - * @param col name of column - * @return column famile as a TextSequence based on the passed - * col. If col is reused, make a new Text of - * the result by calling {@link TextSequence#toText()}. - * @throws InvalidColumnNameException + * @param column + * @return New byte array that holds column qualifier suffix. + * @see #parseColumn(byte[]) */ - public static TextSequence extractFamily(final Text col) - throws InvalidColumnNameException { - return extractFamily(col, false); + public static byte [] getQualifier(final byte [] column) { + int index = getFamilyDelimiterIndex(column); + int len = column.length - (index + 1); + byte [] result = new byte[len]; + System.arraycopy(column, index + 1, result, 0, len); + return result; } - + /** - * Extracts the column family name from a column - * For example, returns 'info' if the specified column was 'info:server' - * @param col name of column - * @param withColon set to true if colon separator should be returned - * @return column famile as a TextSequence based on the passed - * col. If col is reused, make a new Text of - * the result by calling {@link TextSequence#toText()}. - * @throws InvalidColumnNameException + * @param c Column name + * @return Return array of size two whose first element has the family + * prefix of passed column c and whose second element is the + * column qualifier. */ - public static TextSequence extractFamily(final Text col, - final boolean withColon) - throws InvalidColumnNameException { - int offset = getColonOffset(col); - // Include ':' in copy? - offset += (withColon)? 1: 0; - if (offset == col.getLength()) { - return new TextSequence(col); + public static byte [][] parseColumn(final byte [] c) { + byte [][] result = new byte [2][]; + int index = getFamilyDelimiterIndex(c); + if (index == -1) { + throw new IllegalArgumentException("Impossible column name: " + c); } - return new TextSequence(col, 0, offset); + result[0] = new byte [index]; + System.arraycopy(c, 0, result[0], 0, index); + int len = c.length - (index + 1); + result[1] = new byte[len]; + System.arraycopy(c, index + 1 /*Skip delimiter*/, result[1], 0, + len); + return result; } /** - * Extracts the column qualifier, the portion that follows the colon (':') - * family/qualifier separator. - * For example, returns 'server' if the specified column was 'info:server' - * @param col name of column - * @return column qualifier as a TextSequence based on the passed - * col. If col is reused, make a new Text of - * the result by calling {@link TextSequence#toText()}. - * @throws InvalidColumnNameException + * @param b + * @return Index of the family-qualifier colon delimiter character in passed + * buffer. */ - public static TextSequence extractQualifier(final Text col) - throws InvalidColumnNameException { - int offset = getColonOffset(col); - if (offset + 1 == col.getLength()) { - return null; + public static int getFamilyDelimiterIndex(final byte [] b) { + if (b == null) { + throw new NullPointerException(); } - return new TextSequence(col, offset + 1); - } - - private static int getColonOffset(final Text col) - throws InvalidColumnNameException { - int offset = -1; - ByteBuffer bb = ByteBuffer.wrap(col.getBytes()); - for (int lastPosition = bb.position(); bb.hasRemaining(); - lastPosition = bb.position()) { - if (Text.bytesToCodePoint(bb) == COLUMN_FAMILY_DELIMITER) { - offset = lastPosition; + int result = -1; + for (int i = 0; i < b.length; i++) { + if (b[i] == COLUMN_FAMILY_DELIMITER) { + result = i; break; } } - if(offset < 0) { - throw new InvalidColumnNameException(col + " is missing the colon " + - "family/qualifier separator"); - } - return offset; + return result; } /** * Returns row and column bytes out of an HStoreKey. * @param hsk Store key. * @return byte array encoding of HStoreKey - * @throws UnsupportedEncodingException */ - public static byte[] getBytes(final HStoreKey hsk) - throws UnsupportedEncodingException { - StringBuilder s = new StringBuilder(hsk.getRow().toString()); - s.append(hsk.getColumn().toString()); - return s.toString().getBytes(HConstants.UTF8_ENCODING); + public static byte[] getBytes(final HStoreKey hsk) { + return Bytes.add(hsk.getRow(), hsk.getColumn()); + } + + // Writable + + /** {@inheritDoc} */ + public void write(DataOutput out) throws IOException { + Bytes.writeByteArray(out, this.row); + Bytes.writeByteArray(out, this.column); + out.writeLong(timestamp); + } + + /** {@inheritDoc} */ + public void readFields(DataInput in) throws IOException { + this.row = Bytes.readByteArray(in); + this.column = Bytes.readByteArray(in); + this.timestamp = in.readLong(); } } \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/HTableDescriptor.java b/src/java/org/apache/hadoop/hbase/HTableDescriptor.java index 22b89551a04..1e75637976b 100644 --- a/src/java/org/apache/hadoop/hbase/HTableDescriptor.java +++ b/src/java/org/apache/hadoop/hbase/HTableDescriptor.java @@ -22,16 +22,14 @@ package org.apache.hadoop.hbase; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; -import java.util.regex.Matcher; -import java.util.regex.Pattern; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.WritableComparable; /** @@ -39,74 +37,98 @@ import org.apache.hadoop.io.WritableComparable; * column families. */ public class HTableDescriptor implements WritableComparable { - /** table descriptor for root table */ - public static final HTableDescriptor rootTableDesc = + /** Table descriptor for -ROOT- catalog table */ + public static final HTableDescriptor ROOT_TABLEDESC = new HTableDescriptor(HConstants.ROOT_TABLE_NAME, new HColumnDescriptor(HConstants.COLUMN_FAMILY, 1, HColumnDescriptor.CompressionType.NONE, false, false, Integer.MAX_VALUE, HConstants.FOREVER, null)); - /** table descriptor for meta table */ - public static final HTableDescriptor metaTableDesc = + /** Table descriptor for .META. catalog table */ + public static final HTableDescriptor META_TABLEDESC = new HTableDescriptor(HConstants.META_TABLE_NAME, new HColumnDescriptor(HConstants.COLUMN_FAMILY, 1, HColumnDescriptor.CompressionType.NONE, false, false, Integer.MAX_VALUE, HConstants.FOREVER, null)); - private boolean rootregion; - private boolean metaregion; - private Text name; - // TODO: Does this need to be a treemap? Can it be a HashMap? - private final TreeMap families; + private boolean rootregion = false; + private boolean metaregion = false; + private byte [] name = HConstants.EMPTY_BYTE_ARRAY; + private String nameAsString = ""; - /* - * Legal table names can only contain 'word characters': - * i.e. [a-zA-Z_0-9-.]. - * Lets be restrictive until a reason to be otherwise. One reason to limit - * characters in table name is to ensure table regions as entries in META - * regions can be found (See HADOOP-1581 'HBASE: Un-openable tablename bug'). - */ - private static final Pattern LEGAL_TABLE_NAME = - Pattern.compile("^[\\w-.]+$"); + // Key is hash of the family name. + private final Map families = + new HashMap(); - /** Used to construct the table descriptors for root and meta tables */ - private HTableDescriptor(Text name, HColumnDescriptor family) { - rootregion = name.equals(HConstants.ROOT_TABLE_NAME); + /** + * Private constructor used internally creating table descriptors for + * catalog tables: e.g. .META. and -ROOT-. + */ + private HTableDescriptor(final byte [] name, HColumnDescriptor family) { + this.name = name.clone(); + this.rootregion = Bytes.equals(name, HConstants.ROOT_TABLE_NAME); this.metaregion = true; - this.name = new Text(name); - this.families = new TreeMap(); - families.put(family.getName(), family); + this.families.put(Bytes.mapKey(family.getName()), family); } /** * Constructs an empty object. * For deserializing an HTableDescriptor instance only. - * @see #HTableDescriptor(String) + * @see #HTableDescriptor(byte[]) */ public HTableDescriptor() { - this.name = new Text(); - this.families = new TreeMap(); + super(); } /** * Constructor. * @param name Table name. * @throws IllegalArgumentException if passed a table name - * that is made of other than 'word' characters: i.e. - * [a-zA-Z_0-9] + * that is made of other than 'word' characters, underscore or period: i.e. + * [a-zA-Z_0-9.]. + * @see HADOOP-1581 HBASE: Un-openable tablename bug */ - public HTableDescriptor(String name) { - this(); - Matcher m = LEGAL_TABLE_NAME.matcher(name); - if (m == null || !m.matches()) { - throw new IllegalArgumentException( - "Table names can only contain 'word characters': i.e. [a-zA-Z_0-9"); - } - this.name.set(name); - this.rootregion = false; - this.metaregion = false; + public HTableDescriptor(final String name) { + this(Bytes.toBytes(name)); + } + + /** + * Constructor. + * @param name Table name. + * @throws IllegalArgumentException if passed a table name + * that is made of other than 'word' characters, underscore or period: i.e. + * [a-zA-Z_0-9.]. + * @see HADOOP-1581 HBASE: Un-openable tablename bug + */ + public HTableDescriptor(final byte [] name) { + this.name = isLegalTableName(name); + this.nameAsString = Bytes.toString(this.name); } + /** + * Check passed buffer is legal user-space table name. + * @param b Table name. + * @return Returns passed b param + * @throws NullPointerException If passed b is null + * @throws IllegalArgumentException if passed a table name + * that is made of other than 'word' characters or underscores: i.e. + * [a-zA-Z_0-9]. + */ + public static byte [] isLegalTableName(final byte [] b) { + if (b == null || b.length <= 0) { + throw new IllegalArgumentException("Name is null or empty"); + } + for (int i = 0; i < b.length; i++) { + if (Character.isLetterOrDigit(b[i]) || b[i] == '_') { + continue; + } + throw new IllegalArgumentException("Illegal character <" + b[i] + ">. " + + "User-space table names can only contain 'word characters':" + + "i.e. [a-zA-Z_0-9]: " + Bytes.toString(b)); + } + return b; + } + /** @return true if this is the root region */ public boolean isRootRegion() { return rootregion; @@ -123,48 +145,47 @@ public class HTableDescriptor implements WritableComparable { } /** @return name of table */ - public Text getName() { + public byte [] getName() { return name; } + /** @return name of table */ + public String getNameAsString() { + return this.nameAsString; + } + /** * Adds a column family. * @param family HColumnDescriptor of familyto add. */ - public void addFamily(HColumnDescriptor family) { - if (family.getName() == null || family.getName().getLength() <= 0) { + public void addFamily(final HColumnDescriptor family) { + if (family.getName() == null || family.getName().length <= 0) { throw new NullPointerException("Family name cannot be null or empty"); } - families.put(family.getName(), family); + this.families.put(Bytes.mapKey(family.getName()), family); } /** * Checks to see if this table contains the given column family - * - * @param family - family name + * @param c Family name or column name. * @return true if the table contains the specified family name */ - public boolean hasFamily(Text family) { - return families.containsKey(family); + public boolean hasFamily(final byte [] c) { + int index = HStoreKey.getFamilyDelimiterIndex(c); + // If index is -1, then presume we were passed a column family name minus + // the colon delimiter. + return families.containsKey(Bytes.mapKey(c, index == -1? c.length: index)); } - /** - * All the column families in this table. - * - * TODO: What is this used for? Seems Dangerous to let people play with our - * private members. - * - * @return map of family members + /** + * @return Name of this table and then a map of all of the column family + * descriptors. + * @see #getNameAsString() */ - public TreeMap families() { - return families; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "name: " + this.name.toString() + ", families: " + this.families; - } + return "name: " + Bytes.toString(this.name) + ", families: " + + this.families.values(); + } /** {@inheritDoc} */ @Override @@ -176,9 +197,9 @@ public class HTableDescriptor implements WritableComparable { @Override public int hashCode() { // TODO: Cache. - int result = this.name.hashCode(); + int result = Bytes.hashCode(this.name); if (this.families != null && this.families.size() > 0) { - for (Map.Entry e: this.families.entrySet()) { + for (HColumnDescriptor e: this.families.values()) { result ^= e.hashCode(); } } @@ -191,7 +212,7 @@ public class HTableDescriptor implements WritableComparable { public void write(DataOutput out) throws IOException { out.writeBoolean(rootregion); out.writeBoolean(metaregion); - name.write(out); + Bytes.writeByteArray(out, name); out.writeInt(families.size()); for(Iterator it = families.values().iterator(); it.hasNext(); ) { @@ -203,13 +224,14 @@ public class HTableDescriptor implements WritableComparable { public void readFields(DataInput in) throws IOException { this.rootregion = in.readBoolean(); this.metaregion = in.readBoolean(); - this.name.readFields(in); + this.name = Bytes.readByteArray(in); + this.nameAsString = Bytes.toString(this.name); int numCols = in.readInt(); - families.clear(); - for(int i = 0; i < numCols; i++) { + this.families.clear(); + for (int i = 0; i < numCols; i++) { HColumnDescriptor c = new HColumnDescriptor(); c.readFields(in); - families.put(c.getName(), c); + this.families.put(Bytes.mapKey(c.getName()), c); } } @@ -218,22 +240,21 @@ public class HTableDescriptor implements WritableComparable { /** {@inheritDoc} */ public int compareTo(Object o) { HTableDescriptor other = (HTableDescriptor) o; - int result = name.compareTo(other.name); - - if(result == 0) { + int result = Bytes.compareTo(this.name, other.name); + if (result == 0) { result = families.size() - other.families.size(); } - if(result == 0 && families.size() != other.families.size()) { + if (result == 0 && families.size() != other.families.size()) { result = Integer.valueOf(families.size()).compareTo( Integer.valueOf(other.families.size())); } - if(result == 0) { - for(Iterator it = families.values().iterator(), + if (result == 0) { + for (Iterator it = families.values().iterator(), it2 = other.families.values().iterator(); it.hasNext(); ) { result = it.next().compareTo(it2.next()); - if(result != 0) { + if (result != 0) { break; } } @@ -244,8 +265,26 @@ public class HTableDescriptor implements WritableComparable { /** * @return Immutable sorted map of families. */ - public SortedMap getFamilies() { - return Collections.unmodifiableSortedMap(this.families); + public Collection getFamilies() { + return Collections.unmodifiableCollection(this.families.values()); + } + + /** + * @param column + * @return Column descriptor for the passed family name or the family on + * passed in column. + */ + public HColumnDescriptor getFamily(final byte [] column) { + return this.families.get(HStoreKey.getFamilyMapKey(column)); + } + + /** + * @param column + * @return Column descriptor for the passed family name or the family on + * passed in column. + */ + public HColumnDescriptor removeFamily(final byte [] column) { + return this.families.remove(HStoreKey.getFamilyMapKey(column)); } /** @@ -253,7 +292,7 @@ public class HTableDescriptor implements WritableComparable { * @param tableName name of table * @return path for table */ - public static Path getTableDir(Path rootdir, Text tableName) { - return new Path(rootdir, tableName.toString()); + public static Path getTableDir(Path rootdir, final byte [] tableName) { + return new Path(rootdir, Bytes.toString(tableName)); } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/LocalHBaseCluster.java b/src/java/org/apache/hadoop/hbase/LocalHBaseCluster.java index 031da4508b4..7794c64034a 100644 --- a/src/java/org/apache/hadoop/hbase/LocalHBaseCluster.java +++ b/src/java/org/apache/hadoop/hbase/LocalHBaseCluster.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.util.Bytes; /** * This class creates a single process HBase cluster. One thread is created for @@ -332,7 +333,9 @@ public class LocalHBaseCluster implements HConstants { LocalHBaseCluster cluster = new LocalHBaseCluster(conf); cluster.startup(); HBaseAdmin admin = new HBaseAdmin(conf); - admin.createTable(new HTableDescriptor(cluster.getClass().getName())); + HTableDescriptor htd = + new HTableDescriptor(Bytes.toBytes(cluster.getClass().getName())); + admin.createTable(htd); cluster.shutdown(); } } diff --git a/src/java/org/apache/hadoop/hbase/NotServingRegionException.java b/src/java/org/apache/hadoop/hbase/NotServingRegionException.java index ffec34fe1fd..5c93ebe9ac8 100644 --- a/src/java/org/apache/hadoop/hbase/NotServingRegionException.java +++ b/src/java/org/apache/hadoop/hbase/NotServingRegionException.java @@ -21,6 +21,8 @@ package org.apache.hadoop.hbase; import java.io.IOException; +import org.apache.hadoop.hbase.util.Bytes; + /** * Thrown by a region server if it is sent a request for a region it is not * serving. @@ -40,5 +42,12 @@ public class NotServingRegionException extends IOException { public NotServingRegionException(String s) { super(s); } - -} + + /** + * Constructor + * @param s message + */ + public NotServingRegionException(final byte [] s) { + super(Bytes.toString(s)); + } +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index 89144ff5444..c17a3037332 100644 --- a/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -25,24 +25,24 @@ import java.util.NoSuchElementException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MasterNotRunningException; +import org.apache.hadoop.hbase.RemoteExceptionHandler; +import org.apache.hadoop.hbase.TableExistsException; +import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.ipc.HMasterInterface; +import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.io.Text; import org.apache.hadoop.ipc.RemoteException; -import org.apache.hadoop.hbase.ipc.HMasterInterface; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.MasterNotRunningException; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.hadoop.hbase.TableExistsException; -import org.apache.hadoop.hbase.RemoteExceptionHandler; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.hbase.io.Cell; - -import org.apache.hadoop.hbase.ipc.HRegionInterface; /** * Provides administrative functions for HBase @@ -50,11 +50,11 @@ import org.apache.hadoop.hbase.ipc.HRegionInterface; public class HBaseAdmin implements HConstants { protected final Log LOG = LogFactory.getLog(this.getClass().getName()); - protected final HConnection connection; - protected final long pause; - protected final int numRetries; - protected volatile HMasterInterface master; - + private final HConnection connection; + private final long pause; + private final int numRetries; + private volatile HMasterInterface master; + /** * Constructor * @@ -86,11 +86,31 @@ public class HBaseAdmin implements HConstants { * @return True if table exists already. * @throws MasterNotRunningException */ - public boolean tableExists(final Text tableName) throws MasterNotRunningException { + public boolean tableExists(final String tableName) + throws MasterNotRunningException { + return tableExists(Bytes.toBytes(tableName)); + } + + /** + * @param tableName Table to check. + * @return True if table exists already. + * @throws MasterNotRunningException + */ + public boolean tableExists(final Text tableName) + throws MasterNotRunningException { + return tableExists(tableName.getBytes()); + } + + /** + * @param tableName Table to check. + * @return True if table exists already. + * @throws MasterNotRunningException + */ + public boolean tableExists(final byte [] tableName) + throws MasterNotRunningException { if (this.master == null) { throw new MasterNotRunningException("master has been shut down"); } - return connection.tableExists(tableName); } @@ -122,8 +142,8 @@ public class HBaseAdmin implements HConstants { */ public void createTable(HTableDescriptor desc) throws IOException { + HTableDescriptor.isLegalTableName(desc.getName()); createTableAsync(desc); - for (int tries = 0; tries < numRetries; tries++) { try { // Wait for new table to come on-line @@ -149,7 +169,7 @@ public class HBaseAdmin implements HConstants { * * @param desc table descriptor for table * - * @throws IllegalArgumentException if the table name is reserved + * @throws IllegalArgumentException Bad table name. * @throws MasterNotRunningException if master is not running * @throws TableExistsException if table already exists (If concurrent * threads, the table may have been created between test-for-existence @@ -161,13 +181,23 @@ public class HBaseAdmin implements HConstants { if (this.master == null) { throw new MasterNotRunningException("master has been shut down"); } - checkReservedTableName(desc.getName()); + HTableDescriptor.isLegalTableName(desc.getName()); try { this.master.createTable(desc); } catch (RemoteException e) { throw RemoteExceptionHandler.decodeRemoteException(e); } } + + /** + * Deletes a table + * + * @param tableName name of table to delete + * @throws IOException + */ + public void deleteTable(final Text tableName) throws IOException { + deleteTable(tableName.getBytes()); + } /** * Deletes a table @@ -175,14 +205,12 @@ public class HBaseAdmin implements HConstants { * @param tableName name of table to delete * @throws IOException */ - public void deleteTable(Text tableName) throws IOException { + public void deleteTable(final byte [] tableName) throws IOException { if (this.master == null) { throw new MasterNotRunningException("master has been shut down"); } - - checkReservedTableName(tableName); + HTableDescriptor.isLegalTableName(tableName); HRegionLocation firstMetaServer = getFirstMetaServerForTable(tableName); - try { this.master.deleteTable(tableName); } catch (RemoteException e) { @@ -204,12 +232,12 @@ public class HBaseAdmin implements HConstants { break; } boolean found = false; - for (Map.Entry e: values.entrySet()) { - if (e.getKey().equals(COL_REGIONINFO)) { + for (Map.Entry e: values.entrySet()) { + if (Bytes.equals(e.getKey(), COL_REGIONINFO)) { info = (HRegionInfo) Writables.getWritable( e.getValue().getValue(), info); - if (info.getTableDesc().getName().equals(tableName)) { + if (Bytes.equals(info.getTableDesc().getName(), tableName)) { found = true; } } @@ -251,12 +279,21 @@ public class HBaseAdmin implements HConstants { * @param tableName name of the table * @throws IOException */ - public void enableTable(Text tableName) throws IOException { + public void enableTable(final Text tableName) throws IOException { + enableTable(tableName.getBytes()); + } + + /** + * Brings a table on-line (enables it) + * + * @param tableName name of the table + * @throws IOException + */ + public void enableTable(final byte [] tableName) throws IOException { if (this.master == null) { throw new MasterNotRunningException("master has been shut down"); } - - checkReservedTableName(tableName); + HTableDescriptor.isLegalTableName(tableName); HRegionLocation firstMetaServer = getFirstMetaServerForTable(tableName); try { @@ -291,8 +328,8 @@ public class HBaseAdmin implements HConstants { break; } valuesfound += 1; - for (Map.Entry e: values.entrySet()) { - if (e.getKey().equals(COL_REGIONINFO)) { + for (Map.Entry e: values.entrySet()) { + if (Bytes.equals(e.getKey(), COL_REGIONINFO)) { info = (HRegionInfo) Writables.getWritable( e.getValue().getValue(), info); @@ -351,17 +388,25 @@ public class HBaseAdmin implements HConstants { * @param tableName name of table * @throws IOException */ - public void disableTable(Text tableName) throws IOException { + public void disableTable(final Text tableName) throws IOException { + disableTable(tableName.getBytes()); + } + + /** + * Disables a table (takes it off-line) If it is being served, the master + * will tell the servers to stop serving it. + * + * @param tableName name of table + * @throws IOException + */ + public void disableTable(final byte [] tableName) throws IOException { if (this.master == null) { throw new MasterNotRunningException("master has been shut down"); } - - checkReservedTableName(tableName); + HTableDescriptor.isLegalTableName(tableName); HRegionLocation firstMetaServer = getFirstMetaServerForTable(tableName); - try { this.master.disableTable(tableName); - } catch (RemoteException e) { throw RemoteExceptionHandler.decodeRemoteException(e); } @@ -379,7 +424,6 @@ public class HBaseAdmin implements HConstants { scannerId = server.openScanner(firstMetaServer.getRegionInfo().getRegionName(), COL_REGIONINFO_ARRAY, tableName, HConstants.LATEST_TIMESTAMP, null); - boolean disabled = false; while (true) { RowResult values = server.next(scannerId); @@ -390,8 +434,8 @@ public class HBaseAdmin implements HConstants { break; } valuesfound += 1; - for (Map.Entry e: values.entrySet()) { - if (e.getKey().equals(COL_REGIONINFO)) { + for (Map.Entry e: values.entrySet()) { + if (Bytes.equals(e.getKey(), COL_REGIONINFO)) { info = (HRegionInfo) Writables.getWritable( e.getValue().getValue(), info); @@ -449,16 +493,26 @@ public class HBaseAdmin implements HConstants { * @param column column descriptor of column to be added * @throws IOException */ - public void addColumn(Text tableName, HColumnDescriptor column) + public void addColumn(final Text tableName, HColumnDescriptor column) + throws IOException { + addColumn(tableName.getBytes(), column); + } + + /** + * Add a column to an existing table + * + * @param tableName name of the table to add column to + * @param column column descriptor of column to be added + * @throws IOException + */ + public void addColumn(final byte [] tableName, HColumnDescriptor column) throws IOException { if (this.master == null) { throw new MasterNotRunningException("master has been shut down"); } - - checkReservedTableName(tableName); + HTableDescriptor.isLegalTableName(tableName); try { this.master.addColumn(tableName, column); - } catch (RemoteException e) { throw RemoteExceptionHandler.decodeRemoteException(e); } @@ -471,16 +525,26 @@ public class HBaseAdmin implements HConstants { * @param columnName name of column to be deleted * @throws IOException */ - public void deleteColumn(Text tableName, Text columnName) + public void deleteColumn(final Text tableName, final Text columnName) + throws IOException { + deleteColumn(tableName.getBytes(), columnName.getBytes()); + } + + /** + * Delete a column from a table + * + * @param tableName name of table + * @param columnName name of column to be deleted + * @throws IOException + */ + public void deleteColumn(final byte [] tableName, final byte [] columnName) throws IOException { if (this.master == null) { throw new MasterNotRunningException("master has been shut down"); } - - checkReservedTableName(tableName); + HTableDescriptor.isLegalTableName(tableName); try { this.master.deleteColumn(tableName, columnName); - } catch (RemoteException e) { throw RemoteExceptionHandler.decodeRemoteException(e); } @@ -494,17 +558,29 @@ public class HBaseAdmin implements HConstants { * @param descriptor new column descriptor to use * @throws IOException */ - public void modifyColumn(Text tableName, Text columnName, + public void modifyColumn(final Text tableName, final Text columnName, + HColumnDescriptor descriptor) + throws IOException { + modifyColumn(tableName.getBytes(), columnName.getBytes(), descriptor); + } + + /** + * Modify an existing column family on a table + * + * @param tableName name of table + * @param columnName name of column to be modified + * @param descriptor new column descriptor to use + * @throws IOException + */ + public void modifyColumn(final byte [] tableName, final byte [] columnName, HColumnDescriptor descriptor) throws IOException { if (this.master == null) { throw new MasterNotRunningException("master has been shut down"); } - - checkReservedTableName(tableName); + HTableDescriptor.isLegalTableName(tableName); try { this.master.modifyColumn(tableName, columnName, descriptor); - } catch (RemoteException e) { throw RemoteExceptionHandler.decodeRemoteException(e); } @@ -519,7 +595,6 @@ public class HBaseAdmin implements HConstants { if (this.master == null) { throw new MasterNotRunningException("master has been shut down"); } - try { this.master.shutdown(); } catch (RemoteException e) { @@ -529,28 +604,12 @@ public class HBaseAdmin implements HConstants { } } - /* - * Verifies that the specified table name is not a reserved name - * @param tableName - the table name to be checked - * @throws IllegalArgumentException - if the table name is reserved - */ - protected void checkReservedTableName(Text tableName) { - if (tableName == null || tableName.getLength() <= 0) { - throw new IllegalArgumentException("Null or empty table name"); - } - if(tableName.charAt(0) == '-' || - tableName.charAt(0) == '.' || - tableName.find(",") != -1) { - throw new IllegalArgumentException(tableName + " is a reserved table name"); - } - } - - private HRegionLocation getFirstMetaServerForTable(Text tableName) + private HRegionLocation getFirstMetaServerForTable(final byte [] tableName) throws IOException { - Text tableKey = new Text(tableName.toString() + ",,99999999999999"); - return connection.locateRegion(META_TABLE_NAME, tableKey); + return connection.locateRegion(META_TABLE_NAME, + HRegionInfo.createRegionName(tableName, null, NINES)); } - + /** * Check to see if HBase is running. Throw an exception if not. * diff --git a/src/java/org/apache/hadoop/hbase/client/HConnection.java b/src/java/org/apache/hadoop/hbase/client/HConnection.java index 88e973f857e..26479ea4fac 100644 --- a/src/java/org/apache/hadoop/hbase/client/HConnection.java +++ b/src/java/org/apache/hadoop/hbase/client/HConnection.java @@ -21,12 +21,11 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.hbase.ipc.HMasterInterface; -import org.apache.hadoop.hbase.MasterNotRunningException; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HServerAddress; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MasterNotRunningException; +import org.apache.hadoop.hbase.ipc.HMasterInterface; import org.apache.hadoop.hbase.ipc.HRegionInterface; /** @@ -47,7 +46,7 @@ public interface HConnection { * @param tableName Table to check. * @return True if table exists already. */ - public boolean tableExists(final Text tableName); + public boolean tableExists(final byte [] tableName); /** * List all the userspace tables. In other words, scan the META table. @@ -70,7 +69,8 @@ public interface HConnection { * question * @throws IOException */ - public HRegionLocation locateRegion(Text tableName, Text row) + public HRegionLocation locateRegion(final byte [] tableName, + final byte [] row) throws IOException; /** @@ -82,7 +82,8 @@ public interface HConnection { * question * @throws IOException */ - public HRegionLocation relocateRegion(Text tableName, Text row) + public HRegionLocation relocateRegion(final byte [] tableName, + final byte [] row) throws IOException; /** @@ -102,7 +103,8 @@ public interface HConnection { * @return Location of row. * @throws IOException */ - HRegionLocation getRegionLocation(Text tableName, Text row, boolean reload) + HRegionLocation getRegionLocation(byte [] tableName, byte [] row, + boolean reload) throws IOException; /** diff --git a/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java b/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java index f1c936b04d4..cb749a03eec 100644 --- a/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java +++ b/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java @@ -31,27 +31,26 @@ import java.util.concurrent.ConcurrentHashMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.ipc.HbaseRPC; -import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.ipc.RemoteException; -import org.apache.hadoop.hbase.ipc.HMasterInterface; -import org.apache.hadoop.hbase.util.SoftSortedMap; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.MasterNotRunningException; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HServerAddress; +import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.LocalHBaseCluster; -import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.NoServerForRegionException; -import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.RemoteExceptionHandler; - -import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.ipc.HMasterInterface; +import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.ipc.HbaseRPC; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.SoftSortedMap; +import org.apache.hadoop.hbase.util.Writables; +import org.apache.hadoop.ipc.RemoteException; /** * A non-instantiable class that manages connections to multiple tables in @@ -68,7 +67,6 @@ public class HConnectionManager implements HConstants { // A Map of master HServerAddress -> connection information for that instance // Note that although the Map is synchronized, the objects it contains // are mutable and hence require synchronized access to them - private static final Map HBASE_INSTANCES = Collections.synchronizedMap(new HashMap()); @@ -126,9 +124,9 @@ public class HConnectionManager implements HConstants { private HRegionLocation rootRegionLocation; - private Map> - cachedRegionLocations = new ConcurrentHashMap>(); + private Map> + cachedRegionLocations = Collections.synchronizedMap( + new HashMap>()); /** * constructor @@ -221,18 +219,18 @@ public class HConnectionManager implements HConstants { } /** {@inheritDoc} */ - public boolean tableExists(final Text tableName) { + public boolean tableExists(final byte [] tableName) { if (tableName == null) { throw new IllegalArgumentException("Table name cannot be null"); } - if (tableName.equals(ROOT_TABLE_NAME) || tableName.equals(META_TABLE_NAME)) { + if (isMetaTableName(tableName)) { return true; } boolean exists = false; try { HTableDescriptor[] tables = listTables(); for (int i = 0; i < tables.length; i++) { - if (tables[i].getName().equals(tableName)) { + if (Bytes.equals(tables[i].getName(), tableName)) { exists = true; } } @@ -241,19 +239,28 @@ public class HConnectionManager implements HConstants { } return exists; } + + /* + * @param n + * @return Truen if passed tablename n is equal to the name + * of a catalog table. + */ + private static boolean isMetaTableName(final byte [] n) { + return Bytes.equals(n, ROOT_TABLE_NAME) || + Bytes.equals(n, META_TABLE_NAME); + } /** {@inheritDoc} */ - public HRegionLocation getRegionLocation(Text tableName, Text row, - boolean reload) throws IOException { - return reload ? - relocateRegion(tableName, row) : - locateRegion(tableName, row); + public HRegionLocation getRegionLocation(final byte [] name, + final byte [] row, boolean reload) + throws IOException { + return reload? relocateRegion(name, row): locateRegion(name, row); } /** {@inheritDoc} */ public HTableDescriptor[] listTables() throws IOException { HashSet uniqueTables = new HashSet(); - Text startRow = EMPTY_START_ROW; + byte [] startRow = EMPTY_START_ROW; // scan over the each meta region do { @@ -273,7 +280,7 @@ public class HConnectionManager implements HConstants { Writables.getHRegionInfo(values.get(COL_REGIONINFO)); // Only examine the rows where the startKey is zero length - if (info.getStartKey().getLength() == 0) { + if (info.getStartKey().length == 0) { uniqueTables.add(info.getTableDesc()); } } @@ -284,32 +291,34 @@ public class HConnectionManager implements HConstants { callable.setClose(); getRegionServerWithRetries(callable); } - } while (startRow.compareTo(LAST_ROW) != 0); + } while (Bytes.compareTo(startRow, LAST_ROW) != 0); return uniqueTables.toArray(new HTableDescriptor[uniqueTables.size()]); } /** {@inheritDoc} */ - public HRegionLocation locateRegion(Text tableName, Text row) + public HRegionLocation locateRegion(final byte [] tableName, + final byte [] row) throws IOException{ return locateRegion(tableName, row, true); } /** {@inheritDoc} */ - public HRegionLocation relocateRegion(Text tableName, Text row) + public HRegionLocation relocateRegion(final byte [] tableName, + final byte [] row) throws IOException{ return locateRegion(tableName, row, false); } - private HRegionLocation locateRegion(Text tableName, Text row, - boolean useCache) + private HRegionLocation locateRegion(final byte [] tableName, + final byte [] row, boolean useCache) throws IOException{ - if (tableName == null || tableName.getLength() == 0) { + if (tableName == null || tableName.length == 0) { throw new IllegalArgumentException( "table name cannot be null or zero length"); } - if (tableName.equals(ROOT_TABLE_NAME)) { + if (Bytes.equals(tableName, ROOT_TABLE_NAME)) { synchronized (rootRegionLock) { // This block guards against two threads trying to find the root // region at the same time. One will go do the find while the @@ -320,7 +329,7 @@ public class HConnectionManager implements HConstants { } return rootRegionLocation; } - } else if (tableName.equals(META_TABLE_NAME)) { + } else if (Bytes.equals(tableName, META_TABLE_NAME)) { synchronized (metaRegionLock) { // This block guards against two threads trying to load the meta // region at the same time. The first will load the meta region and @@ -339,8 +348,8 @@ public class HConnectionManager implements HConstants { * Search one of the meta tables (-ROOT- or .META.) for the HRegionLocation * info that contains the table and row we're seeking. */ - private HRegionLocation locateRegionInMeta(Text parentTable, - Text tableName, Text row, boolean useCache) + private HRegionLocation locateRegionInMeta(final byte [] parentTable, + final byte [] tableName, final byte [] row, boolean useCache) throws IOException{ HRegionLocation location = null; @@ -359,13 +368,9 @@ public class HConnectionManager implements HConstants { // build the key of the meta region we should be looking for. // the extra 9's on the end are necessary to allow "exact" matches // without knowing the precise region names. - Text metaKey = new Text(tableName.toString() + "," - + row.toString() + ",999999999999999"); - - int tries = 0; - while (true) { - tries++; - + byte [] metaKey = HRegionInfo.createRegionName(tableName, row, + HConstants.NINES); + for (int tries = 0; true; tries++) { if (tries >= numRetries) { throw new NoServerForRegionException("Unable to find region for " + row + " after " + numRetries + " tries."); @@ -382,15 +387,15 @@ public class HConnectionManager implements HConstants { metaLocation.getRegionInfo().getRegionName(), metaKey); if (regionInfoRow == null) { - throw new TableNotFoundException("Table '" + tableName + - "' does not exist."); + throw new TableNotFoundException("Table '" + + Bytes.toString(tableName) + "' does not exist."); } Cell value = regionInfoRow.get(COL_REGIONINFO); if (value == null || value.getValue().length == 0) { throw new IOException("HRegionInfo was null or empty in " + - parentTable); + Bytes.toString(parentTable)); } // convert the row result into the HRegionLocation we need! @@ -398,9 +403,9 @@ public class HConnectionManager implements HConstants { value.getValue(), new HRegionInfo()); // possible we got a region of a different table... - if (!regionInfo.getTableDesc().getName().equals(tableName)) { + if (!Bytes.equals(regionInfo.getTableDesc().getName(), tableName)) { throw new TableNotFoundException( - "Table '" + tableName + "' was not found."); + "Table '" + Bytes.toString(tableName) + "' was not found."); } if (regionInfo.isOffline()) { @@ -412,9 +417,9 @@ public class HConnectionManager implements HConstants { Writables.cellToString(regionInfoRow.get(COL_SERVER)); if (serverAddress.equals("")) { - throw new NoServerForRegionException( - "No server address listed in " + parentTable + " for region " - + regionInfo.getRegionName()); + throw new NoServerForRegionException("No server address listed " + + "in " + Bytes.toString(parentTable) + " for region " + + regionInfo.getRegionNameAsString()); } // instantiate the location @@ -452,81 +457,98 @@ public class HConnectionManager implements HConstants { } } - /** - * Search the cache for a location that fits our table and row key. - * Return null if no suitable region is located. TODO: synchronization note - */ - private HRegionLocation getCachedLocation(Text tableName, Text row) { + /* + * Search the cache for a location that fits our table and row key. + * Return null if no suitable region is located. TODO: synchronization note + * + *

TODO: This method during writing consumes 15% of CPU doing lookup + * into the Soft Reference SortedMap. Improve. + * + * @param tableName + * @param row + * @return Null or region location found in cache. + */ + private HRegionLocation getCachedLocation(final byte [] tableName, + final byte [] row) { // find the map of cached locations for this table - SoftSortedMap tableLocations = - cachedRegionLocations.get(tableName); + Integer key = Bytes.mapKey(tableName); + SoftSortedMap tableLocations = + cachedRegionLocations.get(key); // if tableLocations for this table isn't built yet, make one if (tableLocations == null) { - tableLocations = new SoftSortedMap(); - cachedRegionLocations.put(tableName, tableLocations); + tableLocations = new SoftSortedMap(Bytes.BYTES_COMPARATOR); + cachedRegionLocations.put(key, tableLocations); } // start to examine the cache. we can only do cache actions // if there's something in the cache for this table. - if (!tableLocations.isEmpty()) { - if (tableLocations.containsKey(row)) { - HRegionLocation rl = tableLocations.get(row); - if (rl != null && LOG.isDebugEnabled()) { - LOG.debug("Cache hit in table locations for row <" + - row + "> and tableName " + tableName + - ": location server " + rl.getServerAddress() + - ", location region name " + rl.getRegionInfo().getRegionName()); - } - return rl; - } - - // cut the cache so that we only get the part that could contain - // regions that match our key - SoftSortedMap matchingRegions = - tableLocations.headMap(row); + if (tableLocations.isEmpty()) { + return null; + } - // if that portion of the map is empty, then we're done. otherwise, - // we need to examine the cached location to verify that it is - // a match by end key as well. - if (!matchingRegions.isEmpty()) { - HRegionLocation possibleRegion = - matchingRegions.get(matchingRegions.lastKey()); - - // there is a possibility that the reference was garbage collected - // in the instant since we checked isEmpty(). - if (possibleRegion != null) { - Text endKey = possibleRegion.getRegionInfo().getEndKey(); - - // make sure that the end key is greater than the row we're looking - // for, otherwise the row actually belongs in the next region, not - // this one. the exception case is when the endkey is EMPTY_START_ROW, - // signifying that the region we're checking is actually the last - // region in the table. - if (endKey.equals(EMPTY_TEXT) || endKey.compareTo(row) > 0) { - return possibleRegion; - } + HRegionLocation rl = tableLocations.get(row); + if (rl != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Cache hit in table locations for row <" + row + + "> and tableName " + Bytes.toString(tableName) + + ": location server " + rl.getServerAddress() + + ", location region name " + + rl.getRegionInfo().getRegionNameAsString()); + } + return rl; + } + + // Cut the cache so that we only get the part that could contain + // regions that match our key + SoftSortedMap matchingRegions = + tableLocations.headMap(row); + + // if that portion of the map is empty, then we're done. otherwise, + // we need to examine the cached location to verify that it is + // a match by end key as well. + if (!matchingRegions.isEmpty()) { + HRegionLocation possibleRegion = + matchingRegions.get(matchingRegions.lastKey()); + + // there is a possibility that the reference was garbage collected + // in the instant since we checked isEmpty(). + if (possibleRegion != null) { + byte[] endKey = possibleRegion.getRegionInfo().getEndKey(); + + // make sure that the end key is greater than the row we're looking + // for, otherwise the row actually belongs in the next region, not + // this one. the exception case is when the endkey is EMPTY_START_ROW, + // signifying that the region we're checking is actually the last + // region in the table. + if (Bytes.equals(endKey, HConstants.EMPTY_END_ROW) || + Bytes.compareTo(endKey, row) > 0) { + return possibleRegion; } } } - - // passed all the way through, so we got nothin - complete cache miss + + // Passed all the way through, so we got nothin - complete cache miss return null; } /** - * Delete a cached location, if it satisfies the table name and row - * requirements. - */ - private void deleteCachedLocation(Text tableName, Text row){ + * Delete a cached location, if it satisfies the table name and row + * requirements. + */ + private void deleteCachedLocation(final byte [] tableName, + final byte [] row) { // find the map of cached locations for this table - SoftSortedMap tableLocations = - cachedRegionLocations.get(tableName); + Integer key = Bytes.mapKey(tableName); + SoftSortedMap tableLocations = + cachedRegionLocations.get(key); // if tableLocations for this table isn't built yet, make one if (tableLocations == null) { - tableLocations = new SoftSortedMap(); - cachedRegionLocations.put(tableName, tableLocations); + tableLocations = + new SoftSortedMap(Bytes.BYTES_COMPARATOR); + cachedRegionLocations.put(key, tableLocations); } // start to examine the cache. we can only do cache actions @@ -534,7 +556,7 @@ public class HConnectionManager implements HConstants { if (!tableLocations.isEmpty()) { // cut the cache so that we only get the part that could contain // regions that match our key - SoftSortedMap matchingRegions = + SoftSortedMap matchingRegions = tableLocations.headMap(row); // if that portion of the map is empty, then we're done. otherwise, @@ -544,17 +566,17 @@ public class HConnectionManager implements HConstants { HRegionLocation possibleRegion = matchingRegions.get(matchingRegions.lastKey()); - Text endKey = possibleRegion.getRegionInfo().getEndKey(); + byte [] endKey = possibleRegion.getRegionInfo().getEndKey(); // by nature of the map, we know that the start key has to be < // otherwise it wouldn't be in the headMap. - if (endKey.compareTo(row) <= 0) { + if (Bytes.compareTo(endKey, row) <= 0) { // delete any matching entry HRegionLocation rl = tableLocations.remove(matchingRegions.lastKey()); if (rl != null && LOG.isDebugEnabled()) { - LOG.debug("Removed " + rl.getRegionInfo().getRegionName() + - " from cache because of " + row); + LOG.debug("Removed " + rl.getRegionInfo().getRegionNameAsString() + + " from cache because of " + Bytes.toString(row)); } } } @@ -564,17 +586,20 @@ public class HConnectionManager implements HConstants { /** * Put a newly discovered HRegionLocation into the cache. */ - private void cacheLocation(Text tableName, HRegionLocation location){ - Text startKey = location.getRegionInfo().getStartKey(); + private void cacheLocation(final byte [] tableName, + final HRegionLocation location){ + byte [] startKey = location.getRegionInfo().getStartKey(); // find the map of cached locations for this table - SoftSortedMap tableLocations = - cachedRegionLocations.get(tableName); + Integer key = Bytes.mapKey(tableName); + SoftSortedMap tableLocations = + cachedRegionLocations.get(key); // if tableLocations for this table isn't built yet, make one if (tableLocations == null) { - tableLocations = new SoftSortedMap(); - cachedRegionLocations.put(tableName, tableLocations); + tableLocations = + new SoftSortedMap(Bytes.BYTES_COMPARATOR); + cachedRegionLocations.put(key, tableLocations); } // save the HRegionLocation under the startKey @@ -667,9 +692,9 @@ public class HConnectionManager implements HConstants { try { // if this works, then we're good, and we have an acceptable address, // so we can stop doing retries and return the result. - server.getRegionInfo(HRegionInfo.rootRegionInfo.getRegionName()); + server.getRegionInfo(HRegionInfo.ROOT_REGIONINFO.getRegionName()); if (LOG.isDebugEnabled()) { - LOG.debug("Found ROOT " + HRegionInfo.rootRegionInfo); + LOG.debug("Found ROOT " + HRegionInfo.ROOT_REGIONINFO); } break; } catch (IOException e) { @@ -708,7 +733,7 @@ public class HConnectionManager implements HConstants { // return the region location return new HRegionLocation( - HRegionInfo.rootRegionInfo, rootRegionAddress); + HRegionInfo.ROOT_REGIONINFO, rootRegionAddress); } /** {@inheritDoc} */ diff --git a/src/java/org/apache/hadoop/hbase/client/HTable.java b/src/java/org/apache/hadoop/hbase/client/HTable.java index 4805927782a..a2ae123dcd5 100644 --- a/src/java/org/apache/hadoop/hbase/client/HTable.java +++ b/src/java/org/apache/hadoop/hbase/client/HTable.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.io.Text; @@ -52,7 +53,7 @@ public class HTable implements HConstants { protected final Log LOG = LogFactory.getLog(this.getClass()); protected final HConnection connection; - protected final Text tableName; + protected final byte [] tableName; protected final long pause; protected final int numRetries; protected Random rand; @@ -68,7 +69,32 @@ public class HTable implements HConstants { * @param tableName name of the table * @throws IOException */ - public HTable(HBaseConfiguration conf, Text tableName) throws IOException { + public HTable(HBaseConfiguration conf, final Text tableName) + throws IOException { + this(conf, tableName.getBytes()); + } + + /** + * Creates an object to access a HBase table + * + * @param conf configuration object + * @param tableName name of the table + * @throws IOException + */ + public HTable(HBaseConfiguration conf, final String tableName) + throws IOException { + this(conf, Bytes.toBytes(tableName)); + } + + /** + * Creates an object to access a HBase table + * + * @param conf configuration object + * @param tableName name of the table + * @throws IOException + */ + public HTable(HBaseConfiguration conf, final byte [] tableName) + throws IOException { this.connection = HConnectionManager.getConnection(conf); this.tableName = tableName; this.pause = conf.getLong("hbase.client.pause", 10 * 1000); @@ -83,18 +109,18 @@ public class HTable implements HConstants { * @return Location of row. * @throws IOException */ - public HRegionLocation getRegionLocation(Text row) throws IOException { + public HRegionLocation getRegionLocation(final byte [] row) + throws IOException { return connection.getRegionLocation(tableName, row, false); } - /** @return the connection */ public HConnection getConnection() { return connection; } /** @return the table name */ - public Text getTableName() { + public byte [] getTableName() { return this.tableName; } @@ -106,7 +132,7 @@ public class HTable implements HConstants { HTableDescriptor [] metas = this.connection.listTables(); HTableDescriptor result = null; for (int i = 0; i < metas.length; i++) { - if (metas[i].getName().equals(this.tableName)) { + if (Bytes.equals(metas[i].getName(), this.tableName)) { result = metas[i]; break; } @@ -120,12 +146,12 @@ public class HTable implements HConstants { * @throws IOException */ @SuppressWarnings("null") - public Text[] getStartKeys() throws IOException { - List keyList = new ArrayList(); + public byte [][] getStartKeys() throws IOException { + List keyList = new ArrayList(); long scannerId = -1L; - - Text startRow = new Text(tableName.toString() + ",,999999999999999"); + byte [] startRow = + HRegionInfo.createRegionName(this.tableName, null, NINES); HRegionLocation metaLocation = null; HRegionInterface server; @@ -143,7 +169,7 @@ public class HTable implements HConstants { // open a scanner over the meta region scannerId = server.openScanner( metaLocation.getRegionInfo().getRegionName(), - new Text[]{COL_REGIONINFO}, tableName, LATEST_TIMESTAMP, + new byte[][]{COL_REGIONINFO}, tableName, LATEST_TIMESTAMP, null); // iterate through the scanner, accumulating unique region names @@ -157,7 +183,7 @@ public class HTable implements HConstants { info = (HRegionInfo) Writables.getWritable( values.get(COL_REGIONINFO).getValue(), info); - if (!info.getTableDesc().getName().equals(this.tableName)) { + if (!Bytes.equals(info.getTableDesc().getName(), this.tableName)) { break; } @@ -171,14 +197,14 @@ public class HTable implements HConstants { server.close(scannerId); // advance the startRow to the end key of the current region - startRow = metaLocation.getRegionInfo().getEndKey(); + startRow = metaLocation.getRegionInfo().getEndKey(); } catch (IOException e) { // need retry logic? throw e; } - } while (startRow.compareTo(EMPTY_START_ROW) != 0); + } while (Bytes.compareTo(startRow, EMPTY_START_ROW) != 0); - return keyList.toArray(new Text[keyList.size()]); + return keyList.toArray(new byte [keyList.size()][]); } /** @@ -188,15 +214,15 @@ public class HTable implements HConstants { */ @SuppressWarnings("null") public Map getRegionsInfo() throws IOException { - // TODO This code is a near exact copy of getStartKeys. To be refactored HBASE-626 + // TODO This code is a near exact copy of getStartKeys. To be refactored HBASE-626 HashMap regionMap = new HashMap(); - + long scannerId = -1L; - - Text startRow = new Text(tableName.toString() + ",,999999999999999"); + byte [] startRow = + HRegionInfo.createRegionName(this.tableName, null, NINES); HRegionLocation metaLocation = null; HRegionInterface server; - + // scan over the each meta region do { try{ @@ -211,7 +237,7 @@ public class HTable implements HConstants { // open a scanner over the meta region scannerId = server.openScanner( metaLocation.getRegionInfo().getRegionName(), - new Text[]{COL_REGIONINFO}, tableName, LATEST_TIMESTAMP, + new byte [][]{COL_REGIONINFO}, tableName, LATEST_TIMESTAMP, null); // iterate through the scanner, accumulating regions and their regionserver @@ -224,8 +250,8 @@ public class HTable implements HConstants { HRegionInfo info = new HRegionInfo(); info = (HRegionInfo) Writables.getWritable( values.get(COL_REGIONINFO).getValue(), info); - - if (!info.getTableDesc().getName().equals(this.tableName)) { + + if (!Bytes.equals(info.getTableDesc().getName(), this.tableName)) { break; } @@ -239,15 +265,45 @@ public class HTable implements HConstants { server.close(scannerId); // advance the startRow to the end key of the current region - startRow = metaLocation.getRegionInfo().getEndKey(); + startRow = metaLocation.getRegionInfo().getEndKey(); + + // turn the start row into a location + metaLocation = + connection.locateRegion(META_TABLE_NAME, startRow); } catch (IOException e) { // need retry logic? throw e; } - } while (startRow.compareTo(EMPTY_START_ROW) != 0); + } while (Bytes.compareTo(startRow, EMPTY_START_ROW) != 0); return regionMap; } + /** + * Get a single value for the specified row and column + * + * @param row row key + * @param column column name + * @return value for specified row/column + * @throws IOException + */ + public Cell get(final Text row, final Text column) + throws IOException { + return get(row.getBytes(), column.getBytes()); + } + + /** + * Get a single value for the specified row and column + * + * @param row row key + * @param column column name + * @param numVersions - number of versions to retrieve + * @return value for specified row/column + * @throws IOException + */ + public Cell[] get(final Text row, final Text column, int numVersions) + throws IOException { + return get(row.getBytes(), column.getBytes(), numVersions); + } /** * Get a single value for the specified row and column @@ -257,7 +313,8 @@ public class HTable implements HConstants { * @return value for specified row/column * @throws IOException */ - public Cell get(final Text row, final Text column) throws IOException { + public Cell get(final byte [] row, final byte [] column) + throws IOException { return connection.getRegionServerWithRetries( new ServerCallable(connection, tableName, row) { public Cell call() throws IOException { @@ -267,7 +324,7 @@ public class HTable implements HConstants { } ); } - + /** * Get the specified number of versions of the specified row and column * @@ -277,10 +334,10 @@ public class HTable implements HConstants { * @return - array byte values * @throws IOException */ - public Cell[] get(final Text row, final Text column, final int numVersions) + public Cell[] get(final byte [] row, final byte [] column, + final int numVersions) throws IOException { Cell[] values = null; - values = connection.getRegionServerWithRetries( new ServerCallable(connection, tableName, row) { public Cell[] call() throws IOException { @@ -311,11 +368,27 @@ public class HTable implements HConstants { * @return - array of values that match the above criteria * @throws IOException */ - public Cell[] get(final Text row, final Text column, final long timestamp, - final int numVersions) + public Cell[] get(final Text row, final Text column, + final long timestamp, final int numVersions) + throws IOException { + return get(row.getBytes(), column.getBytes(), timestamp, numVersions); + } + + /** + * Get the specified number of versions of the specified row and column with + * the specified timestamp. + * + * @param row - row key + * @param column - column name + * @param timestamp - timestamp + * @param numVersions - number of versions to retrieve + * @return - array of values that match the above criteria + * @throws IOException + */ + public Cell[] get(final byte [] row, final byte [] column, + final long timestamp, final int numVersions) throws IOException { Cell[] values = null; - values = connection.getRegionServerWithRetries( new ServerCallable(connection, tableName, row) { public Cell[] call() throws IOException { @@ -334,15 +407,26 @@ public class HTable implements HConstants { } return null; } - + /** * Get all the data for the specified row at the latest timestamp * * @param row row key - * @return Map of columns to values. Map is empty if row does not exist. + * @return RowResult is empty if row does not exist. * @throws IOException */ - public Map getRow(final Text row) throws IOException { + public RowResult getRow(final Text row) throws IOException { + return getRow(row.getBytes()); + } + + /** + * Get all the data for the specified row at the latest timestamp + * + * @param row row key + * @return RowResult is empty if row does not exist. + * @throws IOException + */ + public RowResult getRow(final byte [] row) throws IOException { return getRow(row, HConstants.LATEST_TIMESTAMP); } @@ -351,10 +435,23 @@ public class HTable implements HConstants { * * @param row row key * @param ts timestamp - * @return Map of columns to values. Map is empty if row does not exist. + * @return RowResult is empty if row does not exist. * @throws IOException */ - public Map getRow(final Text row, final long ts) + public RowResult getRow(final Text row, final long ts) + throws IOException { + return getRow(row.getBytes(), ts); + } + + /** + * Get all the data for the specified row at a specified timestamp + * + * @param row row key + * @param ts timestamp + * @return RowResult is empty if row does not exist. + * @throws IOException + */ + public RowResult getRow(final byte [] row, final long ts) throws IOException { return connection.getRegionServerWithRetries( new ServerCallable(connection, tableName, row) { @@ -365,16 +462,28 @@ public class HTable implements HConstants { } ); } - /** * Get selected columns for the specified row at the latest timestamp * * @param row row key * @param columns Array of column names you want to retrieve. - * @return Map of columns to values. Map is empty if row does not exist. + * @return RowResult is empty if row does not exist. * @throws IOException */ - public Map getRow(final Text row, final Text[] columns) + public RowResult getRow(final Text row, final Text[] columns) + throws IOException { + return getRow(row.getBytes(), Bytes.toByteArrays(columns)); + } + + /** + * Get selected columns for the specified row at the latest timestamp + * + * @param row row key + * @param columns Array of column names you want to retrieve. + * @return RowResult is empty if row does not exist. + * @throws IOException + */ + public RowResult getRow(final byte [] row, final byte [][] columns) throws IOException { return getRow(row, columns, HConstants.LATEST_TIMESTAMP); } @@ -385,10 +494,25 @@ public class HTable implements HConstants { * @param row row key * @param columns Array of column names you want to retrieve. * @param ts timestamp - * @return Map of columns to values. Map is empty if row does not exist. + * @return RowResult is empty if row does not exist. * @throws IOException */ - public Map getRow(final Text row, final Text[] columns, + public RowResult getRow(final Text row, final Text[] columns, + final long ts) + throws IOException { + return getRow(row.getBytes(), Bytes.toByteArrays(columns), ts); + } + + /** + * Get selected columns for the specified row at a specified timestamp + * + * @param row row key + * @param columns Array of column names you want to retrieve. + * @param ts timestamp + * @return RowResult is empty if row does not exist. + * @throws IOException + */ + public RowResult getRow(final byte [] row, final byte [][] columns, final long ts) throws IOException { return connection.getRegionServerWithRetries( @@ -414,7 +538,25 @@ public class HTable implements HConstants { * @return scanner * @throws IOException */ - public Scanner getScanner(Text[] columns, Text startRow) + public Scanner getScanner(final Text [] columns, final Text startRow) + throws IOException { + return getScanner(Bytes.toByteArrays(columns), startRow.getBytes()); + } + + /** + * Get a scanner on the current table starting at the specified row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @return scanner + * @throws IOException + */ + public Scanner getScanner(final byte[][] columns, final byte [] startRow) throws IOException { return getScanner(columns, startRow, HConstants.LATEST_TIMESTAMP, null); } @@ -433,7 +575,7 @@ public class HTable implements HConstants { * @return scanner * @throws IOException */ - public Scanner getScanner(Text[] columns, Text startRow, + public Scanner getScanner(final byte[][] columns, final byte [] startRow, long timestamp) throws IOException { return getScanner(columns, startRow, timestamp, null); @@ -453,7 +595,7 @@ public class HTable implements HConstants { * @return scanner * @throws IOException */ - public Scanner getScanner(Text[] columns, Text startRow, + public Scanner getScanner(final byte[][] columns, final byte [] startRow, RowFilterInterface filter) throws IOException { return getScanner(columns, startRow, HConstants.LATEST_TIMESTAMP, filter); @@ -476,13 +618,13 @@ public class HTable implements HConstants { * @return scanner * @throws IOException */ - public Scanner getScanner(final Text[] columns, - final Text startRow, final Text stopRow) + public Scanner getScanner(final byte [][] columns, + final byte [] startRow, final byte [] stopRow) throws IOException { return getScanner(columns, startRow, stopRow, HConstants.LATEST_TIMESTAMP); } - + /** * Get a scanner on the current table starting at the specified row and * ending just before stopRow. @@ -503,6 +645,31 @@ public class HTable implements HConstants { */ public Scanner getScanner(final Text[] columns, final Text startRow, final Text stopRow, final long timestamp) + throws IOException { + return getScanner(Bytes.toByteArrays(columns), startRow.getBytes(), + stopRow.getBytes(), timestamp); + } + + /** + * Get a scanner on the current table starting at the specified row and + * ending just before stopRow. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param stopRow Row to stop scanning on. Once we hit this row we stop + * returning values; i.e. we return the row before this one but not the + * stopRow itself. + * @param timestamp only return results whose timestamp <= this value + * @return scanner + * @throws IOException + */ + public Scanner getScanner(final byte [][] columns, + final byte [] startRow, final byte [] stopRow, final long timestamp) throws IOException { return getScanner(columns, startRow, timestamp, new WhileMatchRowFilter(new StopRowFilter(stopRow))); @@ -525,9 +692,64 @@ public class HTable implements HConstants { */ public Scanner getScanner(Text[] columns, Text startRow, long timestamp, RowFilterInterface filter) + throws IOException { + return getScanner(Bytes.toByteArrays(columns), startRow.getBytes(), + timestamp, filter); + } + + /** + * Get a scanner on the current table starting at the specified row. + * Return the specified columns. + * + * @param columns columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param startRow starting row in table to scan + * @param timestamp only return results whose timestamp <= this value + * @param filter a row filter using row-key regexp and/or column data filter. + * @return scanner + * @throws IOException + */ + public Scanner getScanner(final byte [][] columns, + final byte [] startRow, long timestamp, RowFilterInterface filter) throws IOException { return new ClientScanner(columns, startRow, timestamp, filter); } + + /** + * Completely delete the row's cells. + * + * @param row Key of the row you want to completely delete. + * @throws IOException + */ + public void deleteAll(final byte [] row) throws IOException { + deleteAll(row, null); + } + + /** + * Completely delete the row's cells. + * + * @param row Key of the row you want to completely delete. + * @throws IOException + */ + public void deleteAll(final byte [] row, final byte [] column) + throws IOException { + deleteAll(row, column, HConstants.LATEST_TIMESTAMP); + } + + /** + * Completely delete the row's cells. + * + * @param row Key of the row you want to completely delete. + * @param ts Delete all cells of the same timestamp or older. + * @throws IOException + */ + public void deleteAll(final byte [] row, final long ts) + throws IOException { + deleteAll(row, null, ts); + } /** * Delete all cells that match the passed row and column. @@ -538,7 +760,7 @@ public class HTable implements HConstants { public void deleteAll(final Text row, final Text column) throws IOException { deleteAll(row, column, LATEST_TIMESTAMP); } - + /** * Delete all cells that match the passed row and column and whose * timestamp is equal-to or older than the passed timestamp. @@ -548,6 +770,19 @@ public class HTable implements HConstants { * @throws IOException */ public void deleteAll(final Text row, final Text column, final long ts) + throws IOException { + deleteAll(row.getBytes(), column.getBytes(), ts); + } + + /** + * Delete all cells that match the passed row and column and whose + * timestamp is equal-to or older than the passed timestamp. + * @param row Row to update + * @param column name of column whose value is to be deleted + * @param ts Delete all cells of the same timestamp or older. + * @throws IOException + */ + public void deleteAll(final byte [] row, final byte [] column, final long ts) throws IOException { connection.getRegionServerWithRetries( new ServerCallable(connection, tableName, row) { @@ -559,57 +794,6 @@ public class HTable implements HConstants { } ); } - - /** - * Completely delete the row's cells of the same timestamp or older. - * - * @param row Key of the row you want to completely delete. - * @param ts Timestamp of cells to delete - * @throws IOException - */ - public void deleteAll(final Text row, final long ts) throws IOException { - connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public Boolean call() throws IOException { - server.deleteAll(location.getRegionInfo().getRegionName(), row, ts); - return null; - } - } - ); - } - - /** - * Completely delete the row's cells. - * - * @param row Key of the row you want to completely delete. - * @throws IOException - */ - public void deleteAll(final Text row) throws IOException { - deleteAll(row, HConstants.LATEST_TIMESTAMP); - } - - /** - * Delete all cells for a row with matching column family with timestamps - * less than or equal to timestamp. - * - * @param row The row to operate on - * @param family The column family to match - * @param timestamp Timestamp to match - * @throws IOException - */ - public void deleteFamily(final Text row, final Text family, - final long timestamp) - throws IOException { - connection.getRegionServerWithRetries( - new ServerCallable(connection, tableName, row) { - public Boolean call() throws IOException { - server.deleteFamily(location.getRegionInfo().getRegionName(), row, - family, timestamp); - return null; - } - } - ); - } /** * Delete all cells for a row with matching column family at all timestamps. @@ -619,7 +803,31 @@ public class HTable implements HConstants { * @throws IOException */ public void deleteFamily(final Text row, final Text family) throws IOException{ - deleteFamily(row, family, HConstants.LATEST_TIMESTAMP); + deleteFamily(row.getBytes(), family.getBytes(), + HConstants.LATEST_TIMESTAMP); + } + + /** + * Delete all cells for a row with matching column family with timestamps + * less than or equal to timestamp. + * + * @param row The row to operate on + * @param family The column family to match + * @param timestamp Timestamp to match + * @throws IOException + */ + public void deleteFamily(final byte [] row, final byte [] family, + final long timestamp) + throws IOException { + connection.getRegionServerWithRetries( + new ServerCallable(connection, tableName, row) { + public Boolean call() throws IOException { + server.deleteFamily(location.getRegionInfo().getRegionName(), row, + family, timestamp); + return null; + } + } + ); } /** @@ -639,15 +847,15 @@ public class HTable implements HConstants { } ); } - + /** * Implements the scanner interface for the HBase client. * If there are multiple regions in a table, this scanner will iterate * through them all. */ private class ClientScanner implements Scanner { - protected Text[] columns; - private Text startRow; + private byte[][] columns; + private byte [] startRow; protected long scanTime; @SuppressWarnings("hiding") private boolean closed = false; @@ -655,15 +863,20 @@ public class HTable implements HConstants { private ScannerCallable callable = null; protected RowFilterInterface filter; - protected ClientScanner(Text[] columns, Text startRow, long timestamp, - RowFilterInterface filter) + protected ClientScanner(final Text [] columns, final Text startRow, + long timestamp, RowFilterInterface filter) throws IOException { + this(Bytes.toByteArrays(columns), startRow.getBytes(), timestamp, + filter); + } + protected ClientScanner(final byte[][] columns, final byte [] startRow, + final long timestamp, final RowFilterInterface filter) + throws IOException { if (LOG.isDebugEnabled()) { - LOG.debug("Creating scanner over " + tableName + " starting at key '" + - startRow + "'"); + LOG.debug("Creating scanner over " + Bytes.toString(tableName) + + " starting at key '" + startRow + "'"); } - // save off the simple parameters this.columns = columns; this.startRow = startRow; @@ -675,7 +888,6 @@ public class HTable implements HConstants { if (filter != null) { filter.validate(columns); } - nextScanner(); } @@ -698,15 +910,15 @@ public class HTable implements HConstants { LOG.debug("Advancing forward from region " + currentRegion); } - Text endKey = currentRegion.getEndKey(); - if (endKey == null || endKey.equals(EMPTY_TEXT)) { + byte [] endKey = currentRegion.getEndKey(); + if (endKey == null || Bytes.equals(endKey, EMPTY_BYTE_ARRAY)) { close(); return false; } } HRegionInfo oldRegion = this.currentRegion; - Text localStartKey = oldRegion == null ? startRow : oldRegion.getEndKey(); + byte [] localStartKey = oldRegion == null? startRow: oldRegion.getEndKey(); if (LOG.isDebugEnabled()) { LOG.debug("Advancing internal scanner to startKey " + localStartKey); diff --git a/src/java/org/apache/hadoop/hbase/client/RetriesExhaustedException.java b/src/java/org/apache/hadoop/hbase/client/RetriesExhaustedException.java index 0fd2a6fd1a5..690726747b5 100644 --- a/src/java/org/apache/hadoop/hbase/client/RetriesExhaustedException.java +++ b/src/java/org/apache/hadoop/hbase/client/RetriesExhaustedException.java @@ -15,9 +15,10 @@ */ package org.apache.hadoop.hbase.client; -import java.io.IOException; +import java.io.IOException; import java.util.List; -import org.apache.hadoop.io.Text; + +import org.apache.hadoop.hbase.util.Bytes; /** * Exception thrown by HTable methods when an attempt to do something (like @@ -33,23 +34,25 @@ public class RetriesExhaustedException extends IOException { * @param numTries The number of tries we made * @param exceptions List of exceptions that failed before giving up */ - public RetriesExhaustedException(String serverName, Text regionName, Text row, + public RetriesExhaustedException(String serverName, final byte [] regionName, + final byte [] row, int numTries, List exceptions) { super(getMessage(serverName, regionName, row, numTries, exceptions)); } - private static String getMessage(String serverName, Text regionName, Text row, + + private static String getMessage(String serverName, final byte [] regionName, + final byte [] row, int numTries, List exceptions) { StringBuilder buffer = new StringBuilder("Trying to contact region server "); buffer.append(serverName); buffer.append(" for region "); - buffer.append(regionName); + buffer.append(Bytes.toString(regionName)); buffer.append(", row '"); - buffer.append(row); + buffer.append(Bytes.toString(row)); buffer.append("', but failed after "); buffer.append(numTries + 1); buffer.append(" attempts.\nExceptions:\n"); - for (Throwable t : exceptions) { buffer.append(t.toString()); buffer.append("\n"); diff --git a/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java b/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java index 34e426b64c4..f0927d7011d 100644 --- a/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java +++ b/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java @@ -25,7 +25,8 @@ import java.io.IOException; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.filter.RowFilterInterface; import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.io.Text; + + /** * Retryable scanner */ @@ -33,12 +34,12 @@ public class ScannerCallable extends ServerCallable { private long scannerId = -1L; private boolean instantiated = false; private boolean closed = false; - private final Text[] columns; + private final byte [][] columns; private final long timestamp; private final RowFilterInterface filter; - ScannerCallable (HConnection connection, Text tableName, Text[] columns, - Text startRow, long timestamp, RowFilterInterface filter) { + ScannerCallable (HConnection connection, byte [] tableName, byte [][] columns, + byte [] startRow, long timestamp, RowFilterInterface filter) { super(connection, tableName, startRow); this.columns = columns; this.timestamp = timestamp; @@ -89,4 +90,4 @@ public class ScannerCallable extends ServerCallable { } return location.getRegionInfo(); } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/client/ServerCallable.java b/src/java/org/apache/hadoop/hbase/client/ServerCallable.java index be35c3ff433..52757f038af 100644 --- a/src/java/org/apache/hadoop/hbase/client/ServerCallable.java +++ b/src/java/org/apache/hadoop/hbase/client/ServerCallable.java @@ -25,7 +25,6 @@ import java.util.concurrent.Callable; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.io.Text; /** * Implements Callable, used to define the particular actions you would @@ -35,8 +34,8 @@ import org.apache.hadoop.io.Text; */ public abstract class ServerCallable implements Callable { protected final HConnection connection; - protected final Text tableName; - protected final Text row; + protected final byte [] tableName; + protected final byte [] row; protected HRegionLocation location; protected HRegionInterface server; @@ -45,7 +44,7 @@ public abstract class ServerCallable implements Callable { * @param tableName * @param row */ - public ServerCallable(HConnection connection, Text tableName, Text row) { + public ServerCallable(HConnection connection, byte [] tableName, byte [] row) { this.connection = connection; this.tableName = tableName; this.row = row; @@ -67,12 +66,12 @@ public abstract class ServerCallable implements Callable { } /** @return the region name */ - public Text getRegionName() { + public byte [] getRegionName() { return location.getRegionInfo().getRegionName(); } /** @return the row */ - public Text getRow() { + public byte [] getRow() { return row; } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java b/src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java index b71440ffdfa..6b95ea1933b 100644 --- a/src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java +++ b/src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java @@ -19,7 +19,7 @@ */ package org.apache.hadoop.hbase.filter; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.util.Bytes; /** * Subclass of StopRowFilter that filters rows > the stop row, @@ -37,20 +37,19 @@ public class InclusiveStopRowFilter extends StopRowFilter{ * * @param stopRowKey rowKey to filter on. */ - public InclusiveStopRowFilter(final Text stopRowKey) { + public InclusiveStopRowFilter(final byte [] stopRowKey) { super(stopRowKey); } /** {@inheritDoc} */ @Override - public boolean filterRowKey(final Text rowKey) { + public boolean filterRowKey(final byte [] rowKey) { if (rowKey == null) { - if (this.stopRowKey == null) { + if (getStopRowKey() == null) { return true; } return false; } - return this.stopRowKey.compareTo(rowKey) < 0; + return Bytes.compareTo(getStopRowKey(), rowKey) < 0; } - } diff --git a/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java b/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java index 6b7d2597ed4..9f411b8dc6c 100644 --- a/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java +++ b/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java @@ -24,7 +24,6 @@ import java.io.DataOutput; import java.io.IOException; import java.util.SortedMap; -import org.apache.hadoop.io.Text; /** * Implementation of RowFilterInterface that limits results to a specific page @@ -65,7 +64,7 @@ public class PageRowFilter implements RowFilterInterface { * * {@inheritDoc} */ - public void validate(@SuppressWarnings("unused") final Text[] columns) { + public void validate(@SuppressWarnings("unused") final byte [][] columns) { // Doesn't filter columns } @@ -79,7 +78,7 @@ public class PageRowFilter implements RowFilterInterface { /** {@inheritDoc} */ public void rowProcessed(boolean filtered, - @SuppressWarnings("unused") Text rowKey) { + @SuppressWarnings("unused") byte [] rowKey) { if (!filtered) { this.rowsAccepted++; } @@ -105,7 +104,7 @@ public class PageRowFilter implements RowFilterInterface { * * {@inheritDoc} */ - public boolean filterRowKey(@SuppressWarnings("unused") final Text rowKey) { + public boolean filterRowKey(@SuppressWarnings("unused") final byte [] r) { return filterAllRemaining(); } @@ -113,8 +112,8 @@ public class PageRowFilter implements RowFilterInterface { * * {@inheritDoc} */ - public boolean filterColumn(@SuppressWarnings("unused") final Text rowKey, - @SuppressWarnings("unused") final Text colKey, + public boolean filterColumn(@SuppressWarnings("unused") final byte [] rowKey, + @SuppressWarnings("unused") final byte [] colKey, @SuppressWarnings("unused") final byte[] data) { return filterAllRemaining(); } @@ -124,7 +123,7 @@ public class PageRowFilter implements RowFilterInterface { * {@inheritDoc} */ public boolean filterRow(@SuppressWarnings("unused") - final SortedMap columns) { + final SortedMap columns) { return filterAllRemaining(); } diff --git a/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java b/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java index 4e6824d455b..905233a0eb4 100644 --- a/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java +++ b/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java @@ -23,17 +23,16 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.SortedMap; +import java.util.TreeMap; +import java.util.TreeSet; import java.util.Map.Entry; import java.util.regex.Pattern; -import org.apache.hadoop.io.Text; - import org.apache.hadoop.hbase.regionserver.HLogEdit; +import org.apache.hadoop.hbase.util.Bytes; /** * Implementation of RowFilterInterface that can filter by rowkey regular @@ -45,8 +44,10 @@ public class RegExpRowFilter implements RowFilterInterface { private Pattern rowKeyPattern = null; private String rowKeyRegExp = null; - private Map equalsMap = new HashMap(); - private Set nullColumns = new HashSet(); + private Map equalsMap = + new TreeMap(Bytes.BYTES_COMPARATOR); + private Set nullColumns = + new TreeSet(Bytes.BYTES_COMPARATOR); /** * Default constructor, filters nothing. Required though for RPC @@ -72,14 +73,14 @@ public class RegExpRowFilter implements RowFilterInterface { * @param columnFilter */ public RegExpRowFilter(final String rowKeyRegExp, - final Map columnFilter) { + final Map columnFilter) { this.rowKeyRegExp = rowKeyRegExp; this.setColumnFilters(columnFilter); } /** {@inheritDoc} */ @SuppressWarnings("unused") - public void rowProcessed(boolean filtered, Text rowKey) { + public void rowProcessed(boolean filtered, byte [] rowKey) { //doesn't care } @@ -96,7 +97,7 @@ public class RegExpRowFilter implements RowFilterInterface { * @param value * the value that must equal the stored value. */ - public void setColumnFilter(final Text colKey, final byte[] value) { + public void setColumnFilter(final byte [] colKey, final byte[] value) { if (value == null) { nullColumns.add(colKey); } else { @@ -110,12 +111,12 @@ public class RegExpRowFilter implements RowFilterInterface { * @param columnFilter * Map of columns with value criteria. */ - public void setColumnFilters(final Map columnFilter) { + public void setColumnFilters(final Map columnFilter) { if (null == columnFilter) { nullColumns.clear(); equalsMap.clear(); } else { - for (Entry entry : columnFilter.entrySet()) { + for (Entry entry : columnFilter.entrySet()) { setColumnFilter(entry.getKey(), entry.getValue()); } } @@ -141,18 +142,17 @@ public class RegExpRowFilter implements RowFilterInterface { * * {@inheritDoc} */ - public boolean filterRowKey(final Text rowKey) { - if (filtersByRowKey() && rowKey != null) { - return !getRowKeyPattern().matcher(rowKey.toString()).matches(); - } - return false; + public boolean filterRowKey(final byte [] rowKey) { + return (filtersByRowKey() && rowKey != null)? + !getRowKeyPattern().matcher(Bytes.toString(rowKey)).matches(): + false; } /** * * {@inheritDoc} */ - public boolean filterColumn(final Text rowKey, final Text colKey, + public boolean filterColumn(final byte [] rowKey, final byte [] colKey, final byte[] data) { if (filterRowKey(rowKey)) { return true; @@ -175,14 +175,14 @@ public class RegExpRowFilter implements RowFilterInterface { * * {@inheritDoc} */ - public boolean filterRow(final SortedMap columns) { - for (Entry col : columns.entrySet()) { + public boolean filterRow(final SortedMap columns) { + for (Entry col : columns.entrySet()) { if (nullColumns.contains(col.getKey()) && !HLogEdit.isDeleted(col.getValue())) { return true; } } - for (Text col : equalsMap.keySet()) { + for (byte [] col : equalsMap.keySet()) { if (!columns.containsKey(col)) { return true; } @@ -225,8 +225,7 @@ public class RegExpRowFilter implements RowFilterInterface { equalsMap.clear(); int size = in.readInt(); for (int i = 0; i < size; i++) { - Text key = new Text(); - key.readFields(in); + byte [] key = Bytes.readByteArray(in); int len = in.readInt(); byte[] value = null; if (len >= 0) { @@ -239,9 +238,7 @@ public class RegExpRowFilter implements RowFilterInterface { nullColumns.clear(); size = in.readInt(); for (int i = 0; i < size; i++) { - Text key = new Text(); - key.readFields(in); - setColumnFilter(key, null); + setColumnFilter(Bytes.readByteArray(in), null); } } @@ -249,12 +246,12 @@ public class RegExpRowFilter implements RowFilterInterface { * * {@inheritDoc} */ - public void validate(final Text[] columns) { - Set invalids = new HashSet(); - for (Text colKey : getFilterColumns()) { + public void validate(final byte [][] columns) { + Set invalids = new TreeSet(Bytes.BYTES_COMPARATOR); + for (byte [] colKey : getFilterColumns()) { boolean found = false; - for (Text col : columns) { - if (col.equals(colKey)) { + for (byte [] col : columns) { + if (Bytes.equals(col, colKey)) { found = true; break; } @@ -271,8 +268,8 @@ public class RegExpRowFilter implements RowFilterInterface { } } - private Set getFilterColumns() { - Set cols = new HashSet(); + private Set getFilterColumns() { + Set cols = new TreeSet(Bytes.BYTES_COMPARATOR); cols.addAll(equalsMap.keySet()); cols.addAll(nullColumns); return cols; @@ -292,8 +289,8 @@ public class RegExpRowFilter implements RowFilterInterface { // equalsMap out.writeInt(equalsMap.size()); - for (Entry entry : equalsMap.entrySet()) { - entry.getKey().write(out); + for (Entry entry : equalsMap.entrySet()) { + Bytes.writeByteArray(out, entry.getKey()); byte[] value = entry.getValue(); out.writeInt(value.length); out.write(value); @@ -301,8 +298,8 @@ public class RegExpRowFilter implements RowFilterInterface { // null columns out.writeInt(nullColumns.size()); - for (Text col : nullColumns) { - col.write(out); + for (byte [] col : nullColumns) { + Bytes.writeByteArray(out, col); } } } \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java b/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java index 37a0e18481a..fc32e129589 100644 --- a/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java +++ b/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java @@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.filter; import java.util.SortedMap; -import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; /** @@ -48,7 +47,7 @@ public interface RowFilterInterface extends Writable { * @param filtered * @param key */ - void rowProcessed(boolean filtered, Text key); + void rowProcessed(boolean filtered, byte [] key); /** * Returns whether or not the filter should always be processed in any @@ -79,7 +78,7 @@ public interface RowFilterInterface extends Writable { * @param rowKey * @return true if given row key is filtered and row should not be processed. */ - boolean filterRowKey(final Text rowKey); + boolean filterRowKey(final byte [] rowKey); /** * Filters on row key, column name, and column value. This will take individual columns out of a row, @@ -90,7 +89,8 @@ public interface RowFilterInterface extends Writable { * @param columnValue column value to filter on * @return true if row filtered and should not be processed. */ - boolean filterColumn(final Text rowKey, final Text colunmName, final byte[] columnValue); + boolean filterColumn(final byte [] rowKey, final byte [] colunmName, + final byte[] columnValue); /** * Filter on the fully assembled row. This is the last chance to stop a row. @@ -98,7 +98,7 @@ public interface RowFilterInterface extends Writable { * @param columns * @return true if row filtered and should not be processed. */ - boolean filterRow(final SortedMap columns); + boolean filterRow(final SortedMap columns); /** * Validates that this filter applies only to a subset of the given columns. @@ -111,5 +111,5 @@ public interface RowFilterInterface extends Writable { * * @param columns */ - void validate(final Text[] columns); + void validate(final byte [][] columns); } \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java b/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java index c86f3e796ee..80294912801 100644 --- a/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java +++ b/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java @@ -29,7 +29,6 @@ import java.util.SortedMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.io.ObjectWritable; -import org.apache.hadoop.io.Text; /** * Implementation of RowFilterInterface that represents a set of RowFilters @@ -81,7 +80,7 @@ public class RowFilterSet implements RowFilterInterface { } /** {@inheritDoc} */ - public void validate(final Text[] columns) { + public void validate(final byte [][] columns) { for (RowFilterInterface filter : filters) { filter.validate(columns); } @@ -95,7 +94,7 @@ public class RowFilterSet implements RowFilterInterface { } /** {@inheritDoc} */ - public void rowProcessed(boolean filtered, Text rowKey) { + public void rowProcessed(boolean filtered, byte [] rowKey) { for (RowFilterInterface filter : filters) { filter.rowProcessed(filtered, rowKey); } @@ -129,7 +128,7 @@ public class RowFilterSet implements RowFilterInterface { } /** {@inheritDoc} */ - public boolean filterRowKey(final Text rowKey) { + public boolean filterRowKey(final byte [] rowKey) { boolean resultFound = false; boolean result = operator == Operator.MUST_PASS_ONE; for (RowFilterInterface filter : filters) { @@ -153,7 +152,7 @@ public class RowFilterSet implements RowFilterInterface { } /** {@inheritDoc} */ - public boolean filterColumn(final Text rowKey, final Text colKey, + public boolean filterColumn(final byte [] rowKey, final byte [] colKey, final byte[] data) { boolean resultFound = false; boolean result = operator == Operator.MUST_PASS_ONE; @@ -180,7 +179,7 @@ public class RowFilterSet implements RowFilterInterface { } /** {@inheritDoc} */ - public boolean filterRow(final SortedMap columns) { + public boolean filterRow(final SortedMap columns) { boolean resultFound = false; boolean result = operator == Operator.MUST_PASS_ONE; for (RowFilterInterface filter : filters) { diff --git a/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java b/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java index 397c41f2e44..af26e8a0a66 100644 --- a/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java +++ b/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java @@ -24,7 +24,7 @@ import java.io.DataOutput; import java.io.IOException; import java.util.SortedMap; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.util.Bytes; /** * Implementation of RowFilterInterface that filters out rows greater than or @@ -32,7 +32,7 @@ import org.apache.hadoop.io.Text; */ public class StopRowFilter implements RowFilterInterface { - protected Text stopRowKey; + private byte [] stopRowKey; /** * Default constructor, filters nothing. Required though for RPC @@ -47,7 +47,7 @@ public class StopRowFilter implements RowFilterInterface { * * @param stopRowKey rowKey to filter on. */ - public StopRowFilter(final Text stopRowKey) { + public StopRowFilter(final byte [] stopRowKey) { this.stopRowKey = stopRowKey; } @@ -56,7 +56,7 @@ public class StopRowFilter implements RowFilterInterface { * * @return the filter's stopRowKey */ - public Text getStopRowKey() { + public byte [] getStopRowKey() { return this.stopRowKey; } @@ -64,7 +64,7 @@ public class StopRowFilter implements RowFilterInterface { * * {@inheritDoc} */ - public void validate(@SuppressWarnings("unused") final Text[] columns) { + public void validate(@SuppressWarnings("unused") final byte [][] columns) { // Doesn't filter columns } @@ -78,7 +78,7 @@ public class StopRowFilter implements RowFilterInterface { /** {@inheritDoc} */ @SuppressWarnings("unused") - public void rowProcessed(boolean filtered, Text rowKey) { + public void rowProcessed(boolean filtered, byte [] rowKey) { // Doesn't care } @@ -93,14 +93,14 @@ public class StopRowFilter implements RowFilterInterface { } /** {@inheritDoc} */ - public boolean filterRowKey(final Text rowKey) { + public boolean filterRowKey(final byte [] rowKey) { if (rowKey == null) { if (this.stopRowKey == null) { return true; } return false; } - return this.stopRowKey.compareTo(rowKey) <= 0; + return Bytes.compareTo(stopRowKey, rowKey) <= 0; } /** @@ -109,8 +109,8 @@ public class StopRowFilter implements RowFilterInterface { * Because StopRowFilter does not examine column information, this method * defaults to calling the rowKey-only version of filter. */ - public boolean filterColumn(@SuppressWarnings("unused") final Text rowKey, - @SuppressWarnings("unused") final Text colKey, + public boolean filterColumn(@SuppressWarnings("unused") final byte [] rowKey, + @SuppressWarnings("unused") final byte [] colKey, @SuppressWarnings("unused") final byte[] data) { return filterRowKey(rowKey); } @@ -123,17 +123,17 @@ public class StopRowFilter implements RowFilterInterface { * @param columns */ public boolean filterRow(@SuppressWarnings("unused") - final SortedMap columns) { + final SortedMap columns) { return filterAllRemaining(); } /** {@inheritDoc} */ public void readFields(DataInput in) throws IOException { - stopRowKey = new Text(in.readUTF()); + this.stopRowKey = Bytes.readByteArray(in); } /** {@inheritDoc} */ public void write(DataOutput out) throws IOException { - out.writeUTF(stopRowKey.toString()); + Bytes.writeByteArray(out, this.stopRowKey); } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java b/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java index cf6a7e5f3db..2b449ddedbf 100644 --- a/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java +++ b/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java @@ -24,7 +24,6 @@ import java.io.DataOutput; import java.io.IOException; import java.util.SortedMap; -import org.apache.hadoop.io.Text; /** * WhileMatchRowFilter is a wrapper filter that filters everything after the @@ -86,20 +85,20 @@ public class WhileMatchRowFilter implements RowFilterInterface { } /** {@inheritDoc} */ - public boolean filterRowKey(final Text rowKey) { + public boolean filterRowKey(final byte [] rowKey) { changeFAR(this.filter.filterRowKey(rowKey)); return filterAllRemaining(); } /** {@inheritDoc} */ - public boolean filterColumn(final Text rowKey, final Text colKey, + public boolean filterColumn(final byte [] rowKey, final byte [] colKey, final byte[] data) { changeFAR(this.filter.filterColumn(rowKey, colKey, data)); return filterAllRemaining(); } /** {@inheritDoc} */ - public boolean filterRow(final SortedMap columns) { + public boolean filterRow(final SortedMap columns) { changeFAR(this.filter.filterRow(columns)); return filterAllRemaining(); } @@ -115,12 +114,12 @@ public class WhileMatchRowFilter implements RowFilterInterface { } /** {@inheritDoc} */ - public void rowProcessed(boolean filtered, Text rowKey) { + public void rowProcessed(boolean filtered, byte [] rowKey) { this.filter.rowProcessed(filtered, rowKey); } /** {@inheritDoc} */ - public void validate(Text[] columns) { + public void validate(final byte [][] columns) { this.filter.validate(columns); } diff --git a/src/java/org/apache/hadoop/hbase/hql/AlterCommand.java b/src/java/org/apache/hadoop/hbase/hql/AlterCommand.java index 2674c0a030d..39dd2da07af 100644 --- a/src/java/org/apache/hadoop/hbase/hql/AlterCommand.java +++ b/src/java/org/apache/hadoop/hbase/hql/AlterCommand.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.BloomFilterDescriptor; @@ -57,7 +58,7 @@ public class AlterCommand extends SchemaModificationCommand { public ReturnMsg execute(HBaseConfiguration conf) { try { HConnection conn = HConnectionManager.getConnection(conf); - if (!conn.tableExists(new Text(this.tableName))) { + if (!conn.tableExists(Bytes.toBytes(this.tableName))) { return new ReturnMsg(0, "'" + this.tableName + "'" + TABLE_NOT_FOUND); } @@ -95,7 +96,7 @@ public class AlterCommand extends SchemaModificationCommand { // get the table descriptor so we can get the old column descriptor HTableDescriptor tDesc = getTableDescByName(admin, tableName); - HColumnDescriptor oldColumnDesc = tDesc.families().get(columnName); + HColumnDescriptor oldColumnDesc = tDesc.getFamily(columnName.getBytes()); // combine the options specified in the shell with the options // from the exiting descriptor to produce the new descriptor @@ -168,11 +169,11 @@ public class AlterCommand extends SchemaModificationCommand { return CommandType.DDL; } - private HTableDescriptor getTableDescByName(HBaseAdmin admin, String tableName) + private HTableDescriptor getTableDescByName(HBaseAdmin admin, String tn) throws IOException { HTableDescriptor[] tables = admin.listTables(); for (HTableDescriptor tDesc : tables) { - if (tDesc.getName().toString().equals(tableName)) { + if (tDesc.getName().toString().equals(tn)) { return tDesc; } } @@ -184,7 +185,7 @@ public class AlterCommand extends SchemaModificationCommand { * instance of HColumnDescriptor representing the column spec, with empty * values drawn from the original as defaults */ - protected HColumnDescriptor getColumnDescriptor(String column, + protected HColumnDescriptor getColumnDescriptor(String c, Map columnSpec, HColumnDescriptor original) throws IllegalArgumentException { initOptions(original); @@ -230,9 +231,10 @@ public class AlterCommand extends SchemaModificationCommand { } } - column = appendDelimiter(column); + c = appendDelimiter(c); - HColumnDescriptor columnDesc = new HColumnDescriptor(new Text(column), + HColumnDescriptor columnDesc = + new HColumnDescriptor(Bytes.toBytes(c), maxVersions, compression, inMemory, blockCacheEnabled, maxLength, timeToLive, bloomFilterDesc); diff --git a/src/java/org/apache/hadoop/hbase/hql/CreateCommand.java b/src/java/org/apache/hadoop/hbase/hql/CreateCommand.java index d7ad54b0bfc..22db07d1aa9 100644 --- a/src/java/org/apache/hadoop/hbase/hql/CreateCommand.java +++ b/src/java/org/apache/hadoop/hbase/hql/CreateCommand.java @@ -24,12 +24,10 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.client.HConnection; -import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.io.Text; /** @@ -50,7 +48,7 @@ public class CreateCommand extends SchemaModificationCommand { return new ReturnMsg(0, "'" + tableName + "' table already exist."); } - HTableDescriptor tableDesc = new HTableDescriptor(tableName.toString()); + HTableDescriptor tableDesc = new HTableDescriptor(tableName.getBytes()); HColumnDescriptor columnDesc = null; Set columns = columnSpecMap.keySet(); for (String column : columns) { diff --git a/src/java/org/apache/hadoop/hbase/hql/DeleteCommand.java b/src/java/org/apache/hadoop/hbase/hql/DeleteCommand.java index f03a81caf09..1f1aed3de87 100644 --- a/src/java/org/apache/hadoop/hbase/hql/DeleteCommand.java +++ b/src/java/org/apache/hadoop/hbase/hql/DeleteCommand.java @@ -56,9 +56,9 @@ public class DeleteCommand extends BasicCommand { HTable hTable = new HTable(conf, tableName); if (rowKey != null) { - BatchUpdate bu = new BatchUpdate(rowKey); + BatchUpdate bu = new BatchUpdate(rowKey.getBytes()); for (Text column : getColumnList(admin, hTable)) { - bu.delete(new Text(column)); + bu.delete(column.getBytes()); } hTable.commit(bu); } else { diff --git a/src/java/org/apache/hadoop/hbase/hql/DescCommand.java b/src/java/org/apache/hadoop/hbase/hql/DescCommand.java index 7c68cca99be..b4d6c2b28c7 100644 --- a/src/java/org/apache/hadoop/hbase/hql/DescCommand.java +++ b/src/java/org/apache/hadoop/hbase/hql/DescCommand.java @@ -59,8 +59,7 @@ public class DescCommand extends BasicCommand { HColumnDescriptor[] columns = null; for (int i = 0; i < tables.length; i++) { if (tables[i].getName().equals(tableName)) { - columns = tables[i].getFamilies().values().toArray( - new HColumnDescriptor[] {}); + columns = tables[i].getFamilies().toArray(new HColumnDescriptor[] {}); break; } } diff --git a/src/java/org/apache/hadoop/hbase/hql/InsertCommand.java b/src/java/org/apache/hadoop/hbase/hql/InsertCommand.java index cfc65dc16db..332a5504ec5 100644 --- a/src/java/org/apache/hadoop/hbase/hql/InsertCommand.java +++ b/src/java/org/apache/hadoop/hbase/hql/InsertCommand.java @@ -61,8 +61,8 @@ public class InsertCommand extends BasicCommand { try { HTable table = new HTable(conf, tableName); BatchUpdate batchUpdate = timestamp == null ? - new BatchUpdate(getRow()) - : new BatchUpdate(getRow(), Long.parseLong(timestamp)); + new BatchUpdate(getRow().getBytes()) + : new BatchUpdate(getRow().getBytes(), Long.parseLong(timestamp)); for (int i = 0; i < values.size(); i++) { Text column = null; @@ -70,7 +70,7 @@ public class InsertCommand extends BasicCommand { column = getColumn(i); else column = new Text(getColumn(i) + ":"); - batchUpdate.put(column, getValue(i)); + batchUpdate.put(column.getBytes(), getValue(i)); } table.commit(batchUpdate); diff --git a/src/java/org/apache/hadoop/hbase/hql/SchemaModificationCommand.java b/src/java/org/apache/hadoop/hbase/hql/SchemaModificationCommand.java index c6d2286ed1b..4a95b3c77fa 100644 --- a/src/java/org/apache/hadoop/hbase/hql/SchemaModificationCommand.java +++ b/src/java/org/apache/hadoop/hbase/hql/SchemaModificationCommand.java @@ -110,7 +110,7 @@ public abstract class SchemaModificationCommand extends BasicCommand { column = appendDelimiter(column); - HColumnDescriptor columnDesc = new HColumnDescriptor(new Text(column), + HColumnDescriptor columnDesc = new HColumnDescriptor(column.getBytes(), maxVersions, compression, inMemory, blockCacheEnabled, maxLength, timeToLive, bloomFilterDesc); diff --git a/src/java/org/apache/hadoop/hbase/hql/SelectCommand.java b/src/java/org/apache/hadoop/hbase/hql/SelectCommand.java index 69a19a43e78..9f0a3902007 100644 --- a/src/java/org/apache/hadoop/hbase/hql/SelectCommand.java +++ b/src/java/org/apache/hadoop/hbase/hql/SelectCommand.java @@ -26,24 +26,24 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.TreeMap; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HStoreKey; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.Shell; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Scanner; import org.apache.hadoop.hbase.filter.RowFilterInterface; import org.apache.hadoop.hbase.filter.StopRowFilter; import org.apache.hadoop.hbase.filter.WhileMatchRowFilter; import org.apache.hadoop.hbase.hql.generated.HQLParser; -import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.Scanner; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Writables; +import org.apache.hadoop.io.Text; /** * Selects values from tables. @@ -106,8 +106,8 @@ public class SelectCommand extends BasicCommand { } private boolean isMetaTable() { - return (tableName.equals(HConstants.ROOT_TABLE_NAME) || tableName - .equals(HConstants.META_TABLE_NAME)) ? true : false; + return (tableName.equals(new Text(HConstants.ROOT_TABLE_NAME)) || + tableName.equals(new Text(HConstants.META_TABLE_NAME))) ? true : false; } private int compoundWherePrint(HTable table, HBaseAdmin admin) { @@ -118,14 +118,14 @@ public class SelectCommand extends BasicCommand { Cell[] result = null; ParsedColumns parsedColumns = getColumns(admin, false); boolean multiple = parsedColumns.isMultiple() || version > 1; - for (Text column : parsedColumns.getColumns()) { + for (byte [] column : parsedColumns.getColumns()) { if (count == 0) { formatter.header(multiple ? HEADER_COLUMN_CELL : null); } if (timestamp != 0) { - result = table.get(rowKey, column, timestamp, version); + result = table.get(rowKey.getBytes(), column, timestamp, version); } else { - result = table.get(rowKey, column, version); + result = table.get(rowKey.getBytes(), column, version); } for (int ii = 0; result != null && ii < result.length; ii++) { if (multiple) { @@ -138,11 +138,11 @@ public class SelectCommand extends BasicCommand { } } } else { - for (Map.Entry e : table.getRow(rowKey).entrySet()) { + for (Map.Entry e : table.getRow(rowKey).entrySet()) { if (count == 0) { formatter.header(isMultiple() ? HEADER_COLUMN_CELL : null); } - Text key = e.getKey(); + byte [] key = e.getKey(); String keyStr = key.toString(); if (!columns.contains(ASTERISK) && !columns.contains(keyStr)) { continue; @@ -167,28 +167,27 @@ public class SelectCommand extends BasicCommand { return 1; } - private String toString(final Text columnName, final byte[] cell) + private String toString(final byte [] columnName, final byte[] cell) throws IOException { String result = null; - if (columnName.equals(HConstants.COL_REGIONINFO) - || columnName.equals(HConstants.COL_SPLITA) - || columnName.equals(HConstants.COL_SPLITA)) { + if (Bytes.equals(columnName, HConstants.COL_REGIONINFO) + || Bytes.equals(columnName, HConstants.COL_SPLITA) + || Bytes.equals(columnName, HConstants.COL_SPLITB)) { result = Writables.getHRegionInfoOrNull(cell).toString(); - } else if (columnName.equals(HConstants.COL_STARTCODE)) { - result = Long.toString(Writables.bytesToLong(cell)); + } else if (Bytes.equals(columnName, HConstants.COL_STARTCODE)) { + result = Long.toString(Bytes.toLong(cell)); } else { - result = Writables.bytesToString(cell); + result = Bytes.toString(cell); } return result; } - private String toString(final Text columnName, final Cell cell) + private String toString(final byte [] columnName, final Cell cell) throws IOException { if (cell == null) { return null; - } else { - return toString(columnName, cell.getValue()); } + return toString(columnName, cell.getValue()); } /** @@ -196,19 +195,19 @@ public class SelectCommand extends BasicCommand { * could return more than one column. */ class ParsedColumns { - private final List cols; + private final List cols; private final boolean isMultiple; - ParsedColumns(final List columns) { + ParsedColumns(final List columns) { this(columns, true); } - ParsedColumns(final List columns, final boolean isMultiple) { + ParsedColumns(final List columns, final boolean isMultiple) { this.cols = columns; this.isMultiple = isMultiple; } - public List getColumns() { + public List getColumns() { return this.cols; } @@ -226,13 +225,14 @@ public class SelectCommand extends BasicCommand { if (timestamp == 0) { scan = table.getScanner(cols, rowKey); } else { - scan = table.getScanner(cols, rowKey, timestamp); + scan = table.getScanner(Bytes.toByteArrays(cols), rowKey.getBytes(), + timestamp); } if (this.stopRow.toString().length() > 0) { RowFilterInterface filter = new WhileMatchRowFilter(new StopRowFilter( - stopRow)); - scan = table.getScanner(cols, rowKey, filter); + stopRow.getBytes())); + scan = table.getScanner(Bytes.toByteArrays(cols), rowKey.getBytes(), filter); } RowResult results = scan.next(); @@ -243,10 +243,10 @@ public class SelectCommand extends BasicCommand { formatter.header((parsedColumns.isMultiple()) ? HEADER : HEADER_ROW_CELL); } - Text r = results.getRow(); + byte [] r = results.getRow(); if (!countFunction) { - for (Text columnKey : results.keySet()) { + for (byte [] columnKey : results.keySet()) { String cellData = toString(columnKey, results.get(columnKey)); if (parsedColumns.isMultiple()) { formatter.row(new String[] { r.toString(), columnKey.toString(), @@ -287,23 +287,26 @@ public class SelectCommand extends BasicCommand { ParsedColumns result = null; try { if (columns.contains(ASTERISK)) { - if (tableName.equals(HConstants.ROOT_TABLE_NAME) - || tableName.equals(HConstants.META_TABLE_NAME)) { + if (tableName.equals(new Text(HConstants.ROOT_TABLE_NAME)) + || tableName.equals(new Text(HConstants.META_TABLE_NAME))) { result = new ParsedColumns(Arrays.asList(HConstants.COLUMN_FAMILY_ARRAY)); } else { HTableDescriptor[] tables = admin.listTables(); for (int i = 0; i < tables.length; i++) { - if (tables[i].getName().equals(tableName)) { - result = new ParsedColumns(new ArrayList(tables[i].families() - .keySet())); + if (tables[i].getNameAsString().equals(tableName.toString())) { + List cols = new ArrayList(); + for (HColumnDescriptor h: tables[i].getFamilies()) { + cols.add(h.getName()); + } + result = new ParsedColumns(cols); break; } } } } else { - List tmpList = new ArrayList(); + List tmpList = new ArrayList(); for (int i = 0; i < columns.size(); i++) { - Text column = null; + byte [] column = null; // Add '$' to column name if we are scanning. Scanners support // regex column names. Adding '$', the column becomes a // regex that does an explicit match on the supplied column name. @@ -311,8 +314,8 @@ public class SelectCommand extends BasicCommand { // default behavior is to fetch all columns that have a matching // column family. column = (columns.get(i).contains(":")) ? new Text(columns.get(i) - + (scanning ? "$" : "")) : new Text(columns.get(i) + ":" - + (scanning ? "$" : "")); + + (scanning ? "$" : "")).getBytes() : new Text(columns.get(i) + ":" + + (scanning ? "$" : "")).getBytes(); tmpList.add(column); } result = new ParsedColumns(tmpList, tmpList.size() > 1); diff --git a/src/java/org/apache/hadoop/hbase/hql/TruncateCommand.java b/src/java/org/apache/hadoop/hbase/hql/TruncateCommand.java index 862f5bc53b5..3bc94437359 100644 --- a/src/java/org/apache/hadoop/hbase/hql/TruncateCommand.java +++ b/src/java/org/apache/hadoop/hbase/hql/TruncateCommand.java @@ -51,8 +51,8 @@ public class TruncateCommand extends BasicCommand { HTableDescriptor[] tables = admin.listTables(); HColumnDescriptor[] columns = null; for (int i = 0; i < tables.length; i++) { - if (tables[i].getName().equals(tableName)) { - columns = tables[i].getFamilies().values().toArray( + if (tables[i].getNameAsString().equals(tableName.toString())) { + columns = tables[i].getFamilies().toArray( new HColumnDescriptor[] {}); break; } @@ -60,7 +60,7 @@ public class TruncateCommand extends BasicCommand { println("Truncating a '" + tableName + "' table ... Please wait."); admin.deleteTable(tableName); // delete the table - HTableDescriptor tableDesc = new HTableDescriptor(tableName.toString()); + HTableDescriptor tableDesc = new HTableDescriptor(tableName.getBytes()); for (int i = 0; i < columns.length; i++) { tableDesc.addFamily(columns[i]); } diff --git a/src/java/org/apache/hadoop/hbase/io/BatchOperation.java b/src/java/org/apache/hadoop/hbase/io/BatchOperation.java index 47e5287065f..a3ed0ed61ec 100644 --- a/src/java/org/apache/hadoop/hbase/io/BatchOperation.java +++ b/src/java/org/apache/hadoop/hbase/io/BatchOperation.java @@ -23,7 +23,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.Writable; /** @@ -36,20 +36,23 @@ import org.apache.hadoop.io.Writable; * @see BatchUpdate */ public class BatchOperation implements Writable { - private Text column; + private byte [] column = null; // A null value defines DELETE operations. - private byte[] value; - - /** Default constructor used by Writable */ + private byte[] value = null; + + /** + * Default constructor + */ public BatchOperation() { - this(new Text()); + this(null); } + /** * Creates a DELETE batch operation. * @param column column name */ - public BatchOperation(final Text column) { + public BatchOperation(final byte [] column) { this(column, null); } @@ -58,7 +61,7 @@ public class BatchOperation implements Writable { * @param column column name * @param value column value. If non-null, this is a PUT operation. */ - public BatchOperation(final Text column, final byte [] value) { + public BatchOperation(final byte [] column, final byte [] value) { this.column = column; this.value = value; } @@ -66,7 +69,7 @@ public class BatchOperation implements Writable { /** * @return the column */ - public Text getColumn() { + public byte [] getColumn() { return this.column; } @@ -90,7 +93,7 @@ public class BatchOperation implements Writable { // In Performance Evaluation sequentialWrite, 70% of object allocations are // done in here. public void readFields(final DataInput in) throws IOException { - this.column.readFields(in); + this.column = Bytes.readByteArray(in); // Is there a value to read? if (in.readBoolean()) { this.value = new byte[in.readInt()]; @@ -99,7 +102,7 @@ public class BatchOperation implements Writable { } public void write(final DataOutput out) throws IOException { - this.column.write(out); + Bytes.writeByteArray(out, this.column); boolean p = isPut(); out.writeBoolean(p); if (p) { diff --git a/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java b/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java index 6618db6ed3c..1b825f39839 100644 --- a/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java +++ b/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java @@ -25,9 +25,10 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; -import org.apache.hadoop.hbase.HConstants; /** * A Writable object that contains a series of BatchOperations @@ -39,16 +40,29 @@ import org.apache.hadoop.hbase.HConstants; public class BatchUpdate implements Writable, Iterable { // the row being updated - private Text row; + private byte [] row = null; // the batched operations - private ArrayList operations; + private ArrayList operations = + new ArrayList(); - private long timestamp; + private long timestamp = HConstants.LATEST_TIMESTAMP; - /** Default constructor - used by Writable. */ + /** + * Default constructor used serializing. + */ public BatchUpdate() { - this(new Text()); + this ((byte [])null); + } + + /** + * Initialize a BatchUpdate operation on a row. Timestamp is assumed to be + * now. + * + * @param row + */ + public BatchUpdate(final Text row) { + this(row, HConstants.LATEST_TIMESTAMP); } /** @@ -57,24 +71,51 @@ public class BatchUpdate implements Writable, Iterable { * * @param row */ - public BatchUpdate(Text row) { + public BatchUpdate(final String row) { + this(Bytes.toBytes(row), HConstants.LATEST_TIMESTAMP); + } + + /** + * Initialize a BatchUpdate operation on a row. Timestamp is assumed to be + * now. + * + * @param row + */ + public BatchUpdate(final byte [] row) { this(row, HConstants.LATEST_TIMESTAMP); } - + /** * Initialize a BatchUpdate operation on a row with a specific timestamp. * * @param row */ - public BatchUpdate(Text row, long timestamp){ + public BatchUpdate(final String row, long timestamp){ + this(Bytes.toBytes(row), timestamp); + } + + /** + * Initialize a BatchUpdate operation on a row with a specific timestamp. + * + * @param row + */ + public BatchUpdate(final Text row, long timestamp){ + this(row.getBytes(), timestamp); + } + + /** + * Initialize a BatchUpdate operation on a row with a specific timestamp. + * + * @param row + */ + public BatchUpdate(final byte [] row, long timestamp){ this.row = row; this.timestamp = timestamp; this.operations = new ArrayList(); } - /** @return the row */ - public Text getRow() { + public byte [] getRow() { return row; } @@ -91,7 +132,7 @@ public class BatchUpdate implements Writable, Iterable { public void setTimestamp(long timestamp) { this.timestamp = timestamp; } - + /** * Change a value for the specified column * @@ -99,20 +140,60 @@ public class BatchUpdate implements Writable, Iterable { * @param val new value for column. Cannot be null (can be empty). */ public synchronized void put(final Text column, final byte val[]) { + put(column.getBytes(), val); + } + + /** + * Change a value for the specified column + * + * @param column column whose value is being set + * @param val new value for column. Cannot be null (can be empty). + */ + public synchronized void put(final String column, final byte val[]) { + put(Bytes.toBytes(column), val); + } + + /** + * Change a value for the specified column + * + * @param column column whose value is being set + * @param val new value for column. Cannot be null (can be empty). + */ + public synchronized void put(final byte [] column, final byte val[]) { if (val == null) { // If null, the PUT becomes a DELETE operation. throw new IllegalArgumentException("Passed value cannot be null"); } operations.add(new BatchOperation(column, val)); } - + /** * Delete the value for a column * Deletes the cell whose row/column/commit-timestamp match those of the * delete. * @param column name of column whose value is to be deleted */ - public synchronized void delete(final Text column) { + public void delete(final Text column) { + delete(column.getBytes()); + } + + /** + * Delete the value for a column + * Deletes the cell whose row/column/commit-timestamp match those of the + * delete. + * @param column name of column whose value is to be deleted + */ + public void delete(final String column) { + delete(Bytes.toBytes(column)); + } + + /** + * Delete the value for a column + * Deletes the cell whose row/column/commit-timestamp match those of the + * delete. + * @param column name of column whose value is to be deleted + */ + public synchronized void delete(final byte [] column) { operations.add(new BatchOperation(column)); } @@ -137,18 +218,18 @@ public class BatchUpdate implements Writable, Iterable { if (this.operations.size() != 0) { this.operations.clear(); } - row.readFields(in); + this.row = Bytes.readByteArray(in); timestamp = in.readLong(); int nOps = in.readInt(); for (int i = 0; i < nOps; i++) { BatchOperation op = new BatchOperation(); op.readFields(in); - operations.add(op); + this.operations.add(op); } } public void write(final DataOutput out) throws IOException { - row.write(out); + Bytes.writeByteArray(out, this.row); out.writeLong(timestamp); out.writeInt(operations.size()); for (BatchOperation op: operations) { diff --git a/src/java/org/apache/hadoop/hbase/io/Cell.java b/src/java/org/apache/hadoop/hbase/io/Cell.java index b2728ddcb17..b1dbb646595 100644 --- a/src/java/org/apache/hadoop/hbase/io/Cell.java +++ b/src/java/org/apache/hadoop/hbase/io/Cell.java @@ -23,6 +23,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.Writable; /** @@ -69,6 +70,11 @@ public class Cell implements Writable { return timestamp; } + @Override + public String toString() { + return "timestamp=" + this.timestamp + ", value=" + + Bytes.toString(this.value); + } // // Writable // @@ -76,15 +82,12 @@ public class Cell implements Writable { /** {@inheritDoc} */ public void readFields(final DataInput in) throws IOException { timestamp = in.readLong(); - int valueSize = in.readInt(); - value = new byte[valueSize]; - in.readFully(value, 0, valueSize); + this.value = Bytes.readByteArray(in); } /** {@inheritDoc} */ public void write(final DataOutput out) throws IOException { out.writeLong(timestamp); - out.writeInt(value.length); - out.write(value); + Bytes.writeByteArray(out, this.value); } } \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/io/HbaseMapWritable.java b/src/java/org/apache/hadoop/hbase/io/HbaseMapWritable.java index 1f8a61db299..1361830d219 100644 --- a/src/java/org/apache/hadoop/hbase/io/HbaseMapWritable.java +++ b/src/java/org/apache/hadoop/hbase/io/HbaseMapWritable.java @@ -26,34 +26,37 @@ import java.util.Collection; import java.util.HashMap; import java.util.Map; import java.util.Set; +import java.util.TreeMap; import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HStoreKey; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; import org.apache.hadoop.util.ReflectionUtils; -import org.apache.hadoop.hbase.HStoreKey; -import org.apache.hadoop.hbase.io.Cell; - /** * A Writable Map. * Like {@link org.apache.hadoop.io.MapWritable} but dumb. It will fail * if passed a Writable it has not already been told about. Its also been - * primed with hbase Writables. + * primed with hbase Writables. Keys are always byte arrays. Thats other + * difference from MapWritable. + * TODO: Have generics enforce V is a subclass of Writable and K is a byte [] + * only. */ -public class HbaseMapWritable implements Map, Writable, - Configurable { +public class HbaseMapWritable +implements Map, Writable, Configurable { private AtomicReference conf = new AtomicReference(); // Static maps of code to class and vice versa. Includes types used in hbase // only. - static final Map> CODE_TO_CLASS = - new HashMap>(); - static final Map, Byte> CLASS_TO_CODE = - new HashMap, Byte>(); + static final Map> CODE_TO_CLASS = + new HashMap>(); + static final Map, Byte> CLASS_TO_CODE = + new HashMap, Byte>(); static { byte code = 0; @@ -61,22 +64,18 @@ public class HbaseMapWritable implements Map, Writable, addToMap(ImmutableBytesWritable.class, code++); addToMap(Text.class, code++); addToMap(Cell.class, code++); + addToMap(byte [].class, code++); } @SuppressWarnings("boxing") - private static void addToMap(final Class clazz, + private static void addToMap(final Class clazz, final byte code) { CLASS_TO_CODE.put(clazz, code); CODE_TO_CLASS.put(code, clazz); } - private Map instance; - - /** Default constructor. */ - public HbaseMapWritable() { - super(); - this.instance = new HashMap(); - } + private Map instance = + new TreeMap(Bytes.BYTES_COMPARATOR); /** @return the conf */ public Configuration getConf() { @@ -104,12 +103,12 @@ public class HbaseMapWritable implements Map, Writable, } /** {@inheritDoc} */ - public Set> entrySet() { + public Set> entrySet() { return instance.entrySet(); } /** {@inheritDoc} */ - public Writable get(Object key) { + public V get(Object key) { return instance.get(key); } @@ -119,43 +118,17 @@ public class HbaseMapWritable implements Map, Writable, } /** {@inheritDoc} */ - public Set keySet() { + public Set keySet() { return instance.keySet(); } - /** {@inheritDoc} */ - @SuppressWarnings("unchecked") - public Writable put(Writable key, Writable value) { - if (!CLASS_TO_CODE.containsKey(key.getClass())) { - throw new NullPointerException("Unsupported class " + - key.getClass() + " cannot be used as a key."); - } - if (!CLASS_TO_CODE.containsKey(value.getClass())) { - throw new NullPointerException("Unsupported class " + - value.getClass() + " cannot be used as a value."); - } - return instance.put(key, value); - } - - /** {@inheritDoc} */ - public void putAll(Map t) { - for (Map.Entry e: t.entrySet()) { - instance.put(e.getKey(), e.getValue()); - } - } - - /** {@inheritDoc} */ - public Writable remove(Object key) { - return instance.remove(key); - } - /** {@inheritDoc} */ public int size() { return instance.size(); } /** {@inheritDoc} */ - public Collection values() { + public Collection values() { return instance.values(); } @@ -176,18 +149,22 @@ public class HbaseMapWritable implements Map, Writable, } return b; } + + @Override + public String toString() { + return this.instance.toString(); + } /** {@inheritDoc} */ public void write(DataOutput out) throws IOException { // Write out the number of entries in the map - out.writeInt(instance.size()); + out.writeInt(this.instance.size()); // Then write out each key/value pair - for (Map.Entry e: instance.entrySet()) { - out.writeByte(getId(e.getKey().getClass())); - e.getKey().write(out); + for (Map.Entry e: instance.entrySet()) { + Bytes.writeByteArray(out, e.getKey()); out.writeByte(getId(e.getValue().getClass())); - e.getValue().write(out); + ((Writable)e.getValue()).write(out); } } @@ -202,16 +179,24 @@ public class HbaseMapWritable implements Map, Writable, // Then read each key/value pair for (int i = 0; i < entries; i++) { - Writable key = (Writable) ReflectionUtils.newInstance(getClass( - in.readByte()), getConf()); - - key.readFields(in); - - Writable value = (Writable) ReflectionUtils.newInstance(getClass( - in.readByte()), getConf()); - + byte [] key = Bytes.readByteArray(in); + Writable value = (Writable)ReflectionUtils. + newInstance(getClass(in.readByte()), getConf()); value.readFields(in); - instance.put(key, value); + V v = (V)value; + this.instance.put(key, v); } } -} + + public void putAll(Map m) { + this.instance.putAll(m); + } + + public V remove(Object key) { + return this.instance.remove(key); + } + + public V put(byte [] key, V value) { + return this.instance.put(key, value); + } +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java b/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java index b54a1c6f9d1..e701d7502bf 100644 --- a/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java +++ b/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java @@ -39,12 +39,14 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.filter.RowFilterInterface; import org.apache.hadoop.hbase.filter.RowFilterSet; import org.apache.hadoop.hbase.io.HbaseMapWritable; +import org.apache.hadoop.io.MapWritable; import org.apache.hadoop.io.ObjectWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableFactories; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.util.Bytes; /** * This is a customized version of the polymorphic hadoop @@ -123,6 +125,7 @@ public class HbaseObjectWritable implements Writable, Configurable { } addToMap(RowResult.class, code++); addToMap(HRegionInfo[].class, code++); + addToMap(MapWritable.class, code++); } private Class declaredClass; @@ -210,17 +213,24 @@ public class HbaseObjectWritable implements Writable, Configurable { Class declaredClass, Configuration conf) throws IOException { + if (instance == null) { // null instance = new NullInstance(declaredClass, conf); declaredClass = Writable.class; } writeClassCode(out, declaredClass); if (declaredClass.isArray()) { // array - int length = Array.getLength(instance); - out.writeInt(length); - for (int i = 0; i < length; i++) { - writeObject(out, Array.get(instance, i), + // If bytearray, just dump it out -- avoid the recursion and + // byte-at-a-time we were previously doing. + if (declaredClass.equals(byte [].class)) { + Bytes.writeByteArray(out, (byte [])instance); + } else { + int length = Array.getLength(instance); + out.writeInt(length); + for (int i = 0; i < length; i++) { + writeObject(out, Array.get(instance, i), declaredClass.getComponentType(), conf); + } } } else if (declaredClass == String.class) { // String Text.writeString(out, (String)instance); @@ -301,10 +311,14 @@ public class HbaseObjectWritable implements Writable, Configurable { throw new IllegalArgumentException("Not a primitive: "+declaredClass); } } else if (declaredClass.isArray()) { // array - int length = in.readInt(); - instance = Array.newInstance(declaredClass.getComponentType(), length); - for (int i = 0; i < length; i++) { - Array.set(instance, i, readObject(in, conf)); + if (declaredClass.equals(byte [].class)) { + instance = Bytes.readByteArray(in); + } else { + int length = in.readInt(); + instance = Array.newInstance(declaredClass.getComponentType(), length); + for (int i = 0; i < length; i++) { + Array.set(instance, i, readObject(in, conf)); + } } } else if (declaredClass == String.class) { // String instance = Text.readString(in); @@ -353,4 +367,4 @@ public class HbaseObjectWritable implements Writable, Configurable { return this.conf; } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/io/RowResult.java b/src/java/org/apache/hadoop/hbase/io/RowResult.java index ddcb7a50114..f34aa5e88cd 100644 --- a/src/java/org/apache/hadoop/hbase/io/RowResult.java +++ b/src/java/org/apache/hadoop/hbase/io/RowResult.java @@ -23,39 +23,41 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; -import java.util.Iterator; +import java.util.Collection; +import java.util.Collections; import java.util.Map; import java.util.Set; -import java.util.HashSet; -import java.util.Collection; +import java.util.TreeSet; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.io.Writable; -public class RowResult implements Writable, Map { - protected Text row; - protected HbaseMapWritable cells; - - /** - * Used by Writable - */ - public RowResult () { - row = new Text(); - cells = new HbaseMapWritable(); +/** + * Holds row name and then a map of columns to cells. + */ +public class RowResult implements Writable, Map { + private byte [] row = null; + private final HbaseMapWritable cells; + + public RowResult() { + this(null, new HbaseMapWritable()); } - + /** * Create a RowResult from a row and Cell map */ - public RowResult (final Text row, final HbaseMapWritable hbw) { + public RowResult (final byte [] row, + final HbaseMapWritable m) { this.row = row; - this.cells = hbw; + this.cells = m; } /** * Get the row for this RowResult */ - public Text getRow() { + public byte [] getRow() { return row; } @@ -63,19 +65,21 @@ public class RowResult implements Writable, Map { // Map interface // - public Cell put(Text key, Cell value) { + public Cell put(@SuppressWarnings("unused") byte [] key, + @SuppressWarnings("unused") Cell value) { throw new UnsupportedOperationException("RowResult is read-only!"); } - public void putAll(Map map) { + @SuppressWarnings("unchecked") + public void putAll(@SuppressWarnings("unused") Map map) { throw new UnsupportedOperationException("RowResult is read-only!"); } public Cell get(Object key) { - return (Cell)cells.get(key); + return (Cell)this.cells.get(key); } - public Cell remove(Object key) { + public Cell remove(@SuppressWarnings("unused") Object key) { throw new UnsupportedOperationException("RowResult is read-only!"); } @@ -83,7 +87,7 @@ public class RowResult implements Writable, Map { return cells.containsKey(key); } - public boolean containsValue(Object value) { + public boolean containsValue(@SuppressWarnings("unused") Object value) { throw new UnsupportedOperationException("Don't support containsValue!"); } @@ -99,20 +103,16 @@ public class RowResult implements Writable, Map { throw new UnsupportedOperationException("RowResult is read-only!"); } - public Set keySet() { - Set result = new HashSet(); - for (Writable w : cells.keySet()) { - result.add((Text)w); + public Set keySet() { + Set result = new TreeSet(Bytes.BYTES_COMPARATOR); + for (byte [] w : cells.keySet()) { + result.add(w); } return result; } - public Set> entrySet() { - Set> result = new HashSet>(); - for (Map.Entry e : cells.entrySet()) { - result.add(new Entry((Text)e.getKey(), (Cell)e.getValue())); - } - return result; + public Set> entrySet() { + return Collections.unmodifiableSet(this.cells.entrySet()); } public Collection values() { @@ -126,25 +126,28 @@ public class RowResult implements Writable, Map { /** * Get the Cell that corresponds to column */ - public Cell get(Text column) { - return (Cell)cells.get(column); + public Cell get(byte [] column) { + return this.cells.get(column); } - public class Entry implements Map.Entry { - private Text row; - private Cell cell; + /** + * Row entry. + */ + public class Entry implements Map.Entry { + private final byte [] column; + private final Cell cell; - Entry(Text row, Cell cell) { - this.row = row; + Entry(byte [] row, Cell cell) { + this.column = row; this.cell = cell; } - public Cell setValue(Cell c) { + public Cell setValue(@SuppressWarnings("unused") Cell c) { throw new UnsupportedOperationException("RowResult is read-only!"); } - public Text getKey() { - return row; + public byte [] getKey() { + return column; } public Cell getValue() { @@ -152,17 +155,51 @@ public class RowResult implements Writable, Map { } } + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("row="); + sb.append(Bytes.toString(this.row)); + sb.append(", cells={"); + boolean moreThanOne = false; + for (Map.Entry e: this.cells.entrySet()) { + if (moreThanOne) { + sb.append(", "); + } else { + moreThanOne = true; + } + sb.append("(column="); + sb.append(Bytes.toString(e.getKey())); + sb.append(", timestamp="); + sb.append(Long.toString(e.getValue().getTimestamp())); + sb.append(", value="); + byte [] v = e.getValue().getValue(); + if (Bytes.equals(e.getKey(), HConstants.COL_REGIONINFO)) { + try { + sb.append(Writables.getHRegionInfo(v).toString()); + } catch (IOException ioe) { + sb.append(ioe.toString()); + } + } else { + sb.append(v); + } + sb.append(")"); + } + sb.append("}"); + return sb.toString(); + } + // // Writable // - + public void readFields(final DataInput in) throws IOException { - row.readFields(in); - cells.readFields(in); + this.row = Bytes.readByteArray(in); + this.cells.readFields(in); } public void write(final DataOutput out) throws IOException { - row.write(out); - cells.write(out); - } -} + Bytes.writeByteArray(out, this.row); + this.cells.write(out); + } +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/io/TextSequence.java b/src/java/org/apache/hadoop/hbase/io/TextSequence.java deleted file mode 100644 index 67482160599..00000000000 --- a/src/java/org/apache/hadoop/hbase/io/TextSequence.java +++ /dev/null @@ -1,175 +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.io; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.WritableComparator; - -/** - * As CharSequence is to String, so is TextSequence to {@link Text} - * (except a TextSequence is a Text whereas a String is a CharSequence). Use - * when you want to conserve on object creation. - * - *

Use with care. If danger that the passed in {@link Text} instance can - * change during the life of this TextSequence, concretize this TextSequence - * by calling {@link #toText()}. - * - *

Equals considers a Text equal if the TextSequence brackets the same bytes. - * - *

TextSequence will not always work as a Text. For instance, the following - * fails Text c = new Text(new TextSequence(new Text("some string"))); - * because the Text constructor accesses private Text data members - * making the new instance from the passed 'Text'. - * - *

TODO: Should this be an Interface as CharSequence is? - */ -public class TextSequence extends Text { - private Text delegatee; - private int start = 0; - private int end = -1; - - public TextSequence() { - super(); - this.delegatee = new Text(); - } - - public TextSequence(final Text d) { - this(d, 0); - } - - public TextSequence(final Text d, final int s) { - this(d, s, d.getLength()); - } - - public TextSequence(final Text d, final int s, final int e) { - this.delegatee = d; - if (s < 0 || s >= d.getLength()) { - throw new IllegalArgumentException("Nonsensical start position " + s); - } - this.start = s; - if (e == -1) { - this.end = this.delegatee.getLength(); - } else if (e <= 0 || e > d.getLength()) { - throw new IllegalArgumentException("Nonsensical start position " + s); - } else { - this.end = e; - } - } - - public int charAt(int position) { - if (position + this.start > this.end || - position + this.start < this.start) { - return -1; - } - return this.delegatee.charAt(start + position); - } - - public int compareTo(Object o) { - if (o instanceof TextSequence) { - TextSequence that = (TextSequence)o; - if (this == that) { - return 0; - } - return WritableComparator.compareBytes(this.delegatee.getBytes(), - this.start, this.getLength(), - that.delegatee.getBytes(), that.start, that.getLength()); - } - // Presume type is Text as super method does. - Text that = (Text)o; - return WritableComparator.compareBytes(this.delegatee.getBytes(), - this.start, this.getLength(), that.getBytes(), 0, that.getLength()); - } - - public boolean equals(Object o) { - return compareTo(o) == 0; - } - - public int find(String what, int s) { - return this.delegatee.find(what, this.start + s) - this.start; - } - - public int find(String what) { - return find(what, 0); - } - - public byte[] getBytes() { - byte [] b = new byte [getLength()]; - System.arraycopy(this.delegatee.getBytes(), this.start, b, 0, getLength()); - return b; - } - - /** - * @return A new Text instance made from the bytes this TextSequence covers. - */ - public Text toText() { - return new Text(getBytes()); - } - - public int getLength() { - return this.end == -1? this.delegatee.getLength(): this.end - this.start; - } - - public int hashCode() { - int hash = 1; - byte [] b = this.delegatee.getBytes(); - for (int i = this.start, length = getLength(); i < length; i++) - hash = (31 * hash) + b[i]; - return hash; - } - - public void set(byte[] utf8, int start, int len) { - this.delegatee.set(utf8, start, len); - } - - public void set(byte[] utf8) { - this.delegatee.set(utf8); - } - - public void set(String string) { - this.delegatee.set(string); - } - - public void set(Text other) { - this.delegatee.set(other); - this.start = 0; - this.end = other.getLength(); - } - - public String toString() { - return this.delegatee.toString().substring(this.start, this.end); - } - - - public void readFields(DataInput in) throws IOException { - this.start = in.readInt(); - this.end = in.readInt(); - this.delegatee.readFields(in); - } - - public void write(DataOutput out) throws IOException { - out.writeInt(this.start); - out.writeInt(this.end); - this.delegatee.write(out); - } -} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java b/src/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java index 575b1480ad8..e959ce2c8dc 100644 --- a/src/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java +++ b/src/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java @@ -19,13 +19,12 @@ */ package org.apache.hadoop.hbase.ipc; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.ipc.VersionedProtocol; -import org.apache.hadoop.hbase.HTableDescriptor; +import java.io.IOException; + import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HServerAddress; - -import java.io.IOException; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.ipc.VersionedProtocol; /** * Clients interact with the HMasterInterface to gain access to meta-level @@ -38,8 +37,9 @@ public interface HMasterInterface extends VersionedProtocol { * Version was incremented to 2 when we brought the hadoop RPC local to hbase * -- HADOOP-2495 and then to 3 when we changed the RPC to send codes instead * of actual class names (HADOOP-2519). + *

Version 4 when we moved to all byte arrays (HBASE-42). */ - public static final long versionID = 3L; + public static final long versionID = 4L; /** @return true if master is available */ public boolean isMasterRunning(); @@ -58,7 +58,7 @@ public interface HMasterInterface extends VersionedProtocol { * @param tableName * @throws IOException */ - public void deleteTable(Text tableName) throws IOException; + public void deleteTable(final byte [] tableName) throws IOException; /** * Adds a column to the specified table @@ -66,7 +66,8 @@ public interface HMasterInterface extends VersionedProtocol { * @param column column descriptor * @throws IOException */ - public void addColumn(Text tableName, HColumnDescriptor column) throws IOException; + public void addColumn(final byte [] tableName, HColumnDescriptor column) + throws IOException; /** * Modifies an existing column on the specified table @@ -75,7 +76,7 @@ public interface HMasterInterface extends VersionedProtocol { * @param descriptor new column descriptor * @throws IOException */ - public void modifyColumn(Text tableName, Text columnName, + public void modifyColumn(final byte [] tableName, final byte [] columnName, HColumnDescriptor descriptor) throws IOException; @@ -86,14 +87,15 @@ public interface HMasterInterface extends VersionedProtocol { * @param columnName * @throws IOException */ - public void deleteColumn(Text tableName, Text columnName) throws IOException; + public void deleteColumn(final byte [] tableName, final byte [] columnName) + throws IOException; /** * Puts the table on-line (only needed if table has been previously taken offline) * @param tableName * @throws IOException */ - public void enableTable(Text tableName) throws IOException; + public void enableTable(final byte [] tableName) throws IOException; /** * Take table offline @@ -101,7 +103,7 @@ public interface HMasterInterface extends VersionedProtocol { * @param tableName * @throws IOException */ - public void disableTable(Text tableName) throws IOException; + public void disableTable(final byte [] tableName) throws IOException; /** * Shutdown an HBase cluster. @@ -114,4 +116,4 @@ public interface HMasterInterface extends VersionedProtocol { * @return address of server that serves the root region */ public HServerAddress findRootRegion(); -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/ipc/HMasterRegionInterface.java b/src/java/org/apache/hadoop/hbase/ipc/HMasterRegionInterface.java index 465dfc59d52..1e7c7ba6c05 100644 --- a/src/java/org/apache/hadoop/hbase/ipc/HMasterRegionInterface.java +++ b/src/java/org/apache/hadoop/hbase/ipc/HMasterRegionInterface.java @@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.ipc; import java.io.IOException; -import org.apache.hadoop.hbase.io.HbaseMapWritable; +import org.apache.hadoop.io.MapWritable; import org.apache.hadoop.ipc.VersionedProtocol; import org.apache.hadoop.hbase.HServerInfo; import org.apache.hadoop.hbase.HMsg; @@ -32,8 +32,11 @@ import org.apache.hadoop.hbase.HRegionInfo; * goings-on and to obtain data-handling instructions from the HMaster. */ public interface HMasterRegionInterface extends VersionedProtocol { - /** Interface version number */ - public static final long versionID = 1L; + /** Interface version number. + * Version 2 was when the regionServerStartup was changed to return a + * MapWritable instead of a HbaseMapWritable. + */ + public static final long versionID = 2L; /** * Called when a region server first starts @@ -42,7 +45,7 @@ public interface HMasterRegionInterface extends VersionedProtocol { * @return Configuration for the regionserver to use: e.g. filesystem, * hbase rootdir, etc. */ - public HbaseMapWritable regionServerStartup(HServerInfo info) throws IOException; + public MapWritable regionServerStartup(HServerInfo info) throws IOException; /** * Called to renew lease, tell master what the region server is doing and to @@ -59,4 +62,4 @@ public interface HMasterRegionInterface extends VersionedProtocol { public HMsg[] regionServerReport(HServerInfo info, HMsg msgs[], HRegionInfo mostLoadedRegions[]) throws IOException; -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java b/src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java index cb973a686b3..afeb4e50d59 100644 --- a/src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java +++ b/src/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.io.Text; import org.apache.hadoop.ipc.VersionedProtocol; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.NotServingRegionException; @@ -35,8 +34,11 @@ import org.apache.hadoop.hbase.NotServingRegionException; * Clients interact with HRegionServers using a handle to the HRegionInterface. */ public interface HRegionInterface extends VersionedProtocol { - /** initial version */ - public static final long versionID = 2L; + /** + * Protocol version. + * Upped to 3 when we went from Text to byte arrays for row and column names. + */ + public static final long versionID = 3L; /** * Get metainfo about an HRegion @@ -45,7 +47,7 @@ public interface HRegionInterface extends VersionedProtocol { * @return HRegionInfo object for region * @throws NotServingRegionException */ - public HRegionInfo getRegionInfo(final Text regionName) + public HRegionInfo getRegionInfo(final byte [] regionName) throws NotServingRegionException; /** @@ -58,7 +60,7 @@ public interface HRegionInterface extends VersionedProtocol { * @return alue for that region/row/column * @throws IOException */ - public Cell get(final Text regionName, final Text row, final Text column) + public Cell get(final byte [] regionName, final byte [] row, final byte [] column) throws IOException; /** @@ -71,8 +73,8 @@ public interface HRegionInterface extends VersionedProtocol { * @return array of values * @throws IOException */ - public Cell[] get(final Text regionName, final Text row, - final Text column, final int numVersions) + public Cell[] get(final byte [] regionName, final byte [] row, + final byte [] column, final int numVersions) throws IOException; /** @@ -87,8 +89,8 @@ public interface HRegionInterface extends VersionedProtocol { * @return array of values * @throws IOException */ - public Cell[] get(final Text regionName, final Text row, - final Text column, final long timestamp, final int numVersions) + public Cell[] get(final byte [] regionName, final byte [] row, + final byte [] column, final long timestamp, final int numVersions) throws IOException; /** @@ -99,7 +101,8 @@ public interface HRegionInterface extends VersionedProtocol { * @return map of values * @throws IOException */ - public RowResult getRow(final Text regionName, final Text row, final long ts) + public RowResult getRow(final byte [] regionName, final byte [] row, + final long ts) throws IOException; /** @@ -111,7 +114,8 @@ public interface HRegionInterface extends VersionedProtocol { * @return map of values * @throws IOException */ - public RowResult getClosestRowBefore(final Text regionName, final Text row) + public RowResult getClosestRowBefore(final byte [] regionName, + final byte [] row) throws IOException; /** @@ -122,8 +126,8 @@ public interface HRegionInterface extends VersionedProtocol { * @return map of values * @throws IOException */ - public RowResult getRow(final Text regionName, final Text row, - final Text[] columns, final long ts) + public RowResult getRow(final byte [] regionName, final byte [] row, + final byte[][] columns, final long ts) throws IOException; /** @@ -134,8 +138,8 @@ public interface HRegionInterface extends VersionedProtocol { * @return map of values * @throws IOException */ - public RowResult getRow(final Text regionName, final Text row, - final Text[] columns) + public RowResult getRow(final byte [] regionName, final byte [] row, + final byte[][] columns) throws IOException; /** @@ -145,7 +149,7 @@ public interface HRegionInterface extends VersionedProtocol { * @param b BatchUpdate * @throws IOException */ - public void batchUpdate(Text regionName, BatchUpdate b) + public void batchUpdate(final byte [] regionName, final BatchUpdate b) throws IOException; /** @@ -158,7 +162,8 @@ public interface HRegionInterface extends VersionedProtocol { * @param timestamp Delete all entries that have this timestamp or older * @throws IOException */ - public void deleteAll(Text regionName, Text row, Text column, long timestamp) + public void deleteAll(byte [] regionName, byte [] row, byte [] column, + long timestamp) throws IOException; /** @@ -170,7 +175,7 @@ public interface HRegionInterface extends VersionedProtocol { * @param timestamp Delete all entries that have this timestamp or older * @throws IOException */ - public void deleteAll(Text regionName, Text row, long timestamp) + public void deleteAll(byte [] regionName, byte [] row, long timestamp) throws IOException; /** @@ -182,7 +187,7 @@ public interface HRegionInterface extends VersionedProtocol { * @param family The column family to match * @param timestamp Timestamp to match */ - public void deleteFamily(Text regionName, Text row, Text family, + public void deleteFamily(byte [] regionName, byte [] row, byte [] family, long timestamp) throws IOException; @@ -207,13 +212,12 @@ public interface HRegionInterface extends VersionedProtocol { * @return scannerId scanner identifier used in other calls * @throws IOException */ - public long openScanner(Text regionName, Text[] columns, Text startRow, - long timestamp, RowFilterInterface filter) + public long openScanner(final byte [] regionName, final byte [][] columns, + final byte []startRow, long timestamp, RowFilterInterface filter) throws IOException; /** * Get the next set of values - * * @param scannerId clientId passed to openScanner * @return map of values * @throws IOException @@ -227,4 +231,4 @@ public interface HRegionInterface extends VersionedProtocol { * @throws IOException */ public void close(long scannerId) throws IOException; -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java b/src/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java index 66238219621..159777540e5 100644 --- a/src/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java +++ b/src/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java @@ -26,8 +26,9 @@ import java.util.Map; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; @@ -36,7 +37,7 @@ import org.apache.hadoop.mapred.Reporter; /** * Extract grouping columns from input record */ -public class GroupingTableMap extends TableMap { +public class GroupingTableMap extends TableMap { /** * JobConf parameter to specify the columns used to produce the key passed to @@ -45,7 +46,7 @@ public class GroupingTableMap extends TableMap { public static final String GROUP_COLUMNS = "hbase.mapred.groupingtablemap.columns"; - protected Text[] m_columns; + protected byte [][] m_columns; /** * Use this before submitting a TableMap job. It will appropriately set up the @@ -62,7 +63,7 @@ public class GroupingTableMap extends TableMap { public static void initJob(String table, String columns, String groupColumns, Class mapper, JobConf job) { - initJob(table, columns, mapper, Text.class, RowResult.class, job); + initJob(table, columns, mapper, ImmutableBytesWritable.class, RowResult.class, job); job.set(GROUP_COLUMNS, groupColumns); } @@ -71,9 +72,9 @@ public class GroupingTableMap extends TableMap { public void configure(JobConf job) { super.configure(job); String[] cols = job.get(GROUP_COLUMNS, "").split(" "); - m_columns = new Text[cols.length]; + m_columns = new byte[cols.length][]; for(int i = 0; i < cols.length; i++) { - m_columns[i] = new Text(cols[i]); + m_columns[i] = Bytes.toBytes(cols[i]); } } @@ -84,13 +85,13 @@ public class GroupingTableMap extends TableMap { * If any of the grouping columns are not found in the value, the record is skipped. */ @Override - public void map(@SuppressWarnings("unused") Text key, - RowResult value, OutputCollector output, + public void map(@SuppressWarnings("unused") ImmutableBytesWritable key, + RowResult value, OutputCollector output, @SuppressWarnings("unused") Reporter reporter) throws IOException { byte[][] keyVals = extractKeyValues(value); if(keyVals != null) { - Text tKey = createGroupKey(keyVals); + ImmutableBytesWritable tKey = createGroupKey(keyVals); output.collect(tKey, value); } } @@ -109,10 +110,10 @@ public class GroupingTableMap extends TableMap { ArrayList foundList = new ArrayList(); int numCols = m_columns.length; if(numCols > 0) { - for (Map.Entry e: r.entrySet()) { - Text column = e.getKey(); + for (Map.Entry e: r.entrySet()) { + byte [] column = e.getKey(); for (int i = 0; i < numCols; i++) { - if (column.equals(m_columns[i])) { + if (Bytes.equals(column, m_columns[i])) { foundList.add(e.getValue().getValue()); break; } @@ -132,7 +133,7 @@ public class GroupingTableMap extends TableMap { * @param vals * @return key generated by concatenating multiple column values */ - protected Text createGroupKey(byte[][] vals) { + protected ImmutableBytesWritable createGroupKey(byte[][] vals) { if(vals == null) { return null; } @@ -147,6 +148,6 @@ public class GroupingTableMap extends TableMap { throw new RuntimeException(e); } } - return new Text(sb.toString()); + return new ImmutableBytesWritable(Bytes.toBytes(sb.toString())); } } diff --git a/src/java/org/apache/hadoop/hbase/mapred/IdentityTableMap.java b/src/java/org/apache/hadoop/hbase/mapred/IdentityTableMap.java index c81d0536c4f..f4b576ec6e5 100644 --- a/src/java/org/apache/hadoop/hbase/mapred/IdentityTableMap.java +++ b/src/java/org/apache/hadoop/hbase/mapred/IdentityTableMap.java @@ -21,8 +21,8 @@ package org.apache.hadoop.hbase.mapred; import java.io.IOException; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; @@ -30,7 +30,7 @@ import org.apache.hadoop.mapred.Reporter; /** * Pass the given key and record as-is to reduce */ -public class IdentityTableMap extends TableMap { +public class IdentityTableMap extends TableMap { /** constructor */ public IdentityTableMap() { @@ -49,15 +49,16 @@ public class IdentityTableMap extends TableMap { @SuppressWarnings("unchecked") public static void initJob(String table, String columns, Class mapper, JobConf job) { - TableMap.initJob(table, columns, mapper, Text.class, RowResult.class, job); + TableMap.initJob(table, columns, mapper, ImmutableBytesWritable.class, + RowResult.class, job); } /** * Pass the key, value to reduce */ @Override - public void map(Text key, RowResult value, - OutputCollector output, + public void map(ImmutableBytesWritable key, RowResult value, + OutputCollector output, @SuppressWarnings("unused") Reporter reporter) throws IOException { // convert diff --git a/src/java/org/apache/hadoop/hbase/mapred/IdentityTableReduce.java b/src/java/org/apache/hadoop/hbase/mapred/IdentityTableReduce.java index 2fd27a3545d..e7f4afb0761 100644 --- a/src/java/org/apache/hadoop/hbase/mapred/IdentityTableReduce.java +++ b/src/java/org/apache/hadoop/hbase/mapred/IdentityTableReduce.java @@ -22,19 +22,16 @@ package org.apache.hadoop.hbase.mapred; import java.io.IOException; import java.util.Iterator; -import org.apache.hadoop.io.MapWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapred.OutputCollector; -import org.apache.hadoop.mapred.Reporter; -import org.apache.hadoop.hbase.io.BatchUpdate; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.io.BatchUpdate;import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hadoop.mapred.Reporter; /** * Write to table each key, record pair */ -public class IdentityTableReduce extends TableReduce { +public class IdentityTableReduce extends TableReduce { private static final Log LOG = LogFactory.getLog(IdentityTableReduce.class.getName()); @@ -44,8 +41,8 @@ public class IdentityTableReduce extends TableReduce { * @see org.apache.hadoop.hbase.mapred.TableReduce#reduce(org.apache.hadoop.io.WritableComparable, java.util.Iterator, org.apache.hadoop.mapred.OutputCollector, org.apache.hadoop.mapred.Reporter) */ @Override - public void reduce(Text key, Iterator values, - OutputCollector output, + public void reduce(ImmutableBytesWritable key, Iterator values, + OutputCollector output, @SuppressWarnings("unused") Reporter reporter) throws IOException { @@ -53,4 +50,4 @@ public class IdentityTableReduce extends TableReduce { output.collect(key, values.next()); } } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/mapred/IndexOutputFormat.java b/src/java/org/apache/hadoop/hbase/mapred/IndexOutputFormat.java index 3721078297a..14c9f0cad2c 100644 --- a/src/java/org/apache/hadoop/hbase/mapred/IndexOutputFormat.java +++ b/src/java/org/apache/hadoop/hbase/mapred/IndexOutputFormat.java @@ -26,7 +26,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputFormatBase; import org.apache.hadoop.mapred.RecordWriter; @@ -42,11 +42,11 @@ import org.apache.lucene.search.Similarity; * the index, and copy the index to the destination. */ public class IndexOutputFormat extends - OutputFormatBase { + OutputFormatBase { static final Log LOG = LogFactory.getLog(IndexOutputFormat.class); @Override - public RecordWriter getRecordWriter( + public RecordWriter getRecordWriter( final FileSystem fs, JobConf job, String name, final Progressable progress) throws IOException { @@ -97,11 +97,11 @@ public class IndexOutputFormat extends } writer.setUseCompoundFile(indexConf.isUseCompoundFile()); - return new RecordWriter() { + return new RecordWriter() { private boolean closed; private long docCount = 0; - public void write(@SuppressWarnings("unused") Text key, + public void write(@SuppressWarnings("unused") ImmutableBytesWritable key, LuceneDocumentWrapper value) throws IOException { // unwrap and index doc diff --git a/src/java/org/apache/hadoop/hbase/mapred/IndexTableReduce.java b/src/java/org/apache/hadoop/hbase/mapred/IndexTableReduce.java index c6222d750a2..b5fbf7ba0fc 100644 --- a/src/java/org/apache/hadoop/hbase/mapred/IndexTableReduce.java +++ b/src/java/org/apache/hadoop/hbase/mapred/IndexTableReduce.java @@ -24,11 +24,10 @@ import java.util.Iterator; import java.util.Map; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.Writable; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.MapReduceBase; import org.apache.hadoop.mapred.OutputCollector; @@ -43,7 +42,7 @@ import org.apache.lucene.document.Field; * to build a Lucene index */ public class IndexTableReduce extends MapReduceBase implements - Reducer { + Reducer { private static final Logger LOG = Logger.getLogger(IndexTableReduce.class); private IndexConfiguration indexConf; @@ -64,9 +63,10 @@ public class IndexTableReduce extends MapReduceBase implements super.close(); } - public void reduce(Text key, Iterator values, - OutputCollector output, Reporter reporter) - throws IOException { + public void reduce(ImmutableBytesWritable key, Iterator values, + OutputCollector output, + Reporter reporter) + throws IOException { if (!values.hasNext()) { return; } @@ -74,7 +74,8 @@ public class IndexTableReduce extends MapReduceBase implements Document doc = new Document(); // index and store row key, row key already UTF-8 encoded - Field keyField = new Field(indexConf.getRowkeyName(), key.toString(), + Field keyField = new Field(indexConf.getRowkeyName(), + Bytes.toString(key.get()), Field.Store.YES, Field.Index.UN_TOKENIZED); keyField.setOmitNorms(true); doc.add(keyField); @@ -83,7 +84,7 @@ public class IndexTableReduce extends MapReduceBase implements RowResult value = values.next(); // each column (name-value pair) is a field (name-value pair) - for (Map.Entry entry : value.entrySet()) { + for (Map.Entry entry : value.entrySet()) { // name is already UTF-8 encoded String column = entry.getKey().toString(); byte[] columnValue = entry.getValue().getValue(); diff --git a/src/java/org/apache/hadoop/hbase/mapred/RowCounter.java b/src/java/org/apache/hadoop/hbase/mapred/RowCounter.java index 89747d7b36d..3e751cf1a23 100644 --- a/src/java/org/apache/hadoop/hbase/mapred/RowCounter.java +++ b/src/java/org/apache/hadoop/hbase/mapred/RowCounter.java @@ -26,8 +26,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputCollector; @@ -41,7 +41,7 @@ import org.apache.hadoop.util.ToolRunner; * Map outputs table rows IF the input row has columns that have content. * Uses an {@link IdentityReducer} */ -public class RowCounter extends TableMap implements Tool { +public class RowCounter extends TableMap implements Tool { /* Name of this 'program' */ static final String NAME = "rowcounter"; @@ -51,12 +51,12 @@ public class RowCounter extends TableMap implements Tool { private static enum Counters {ROWS} @Override - public void map(Text row, RowResult value, - OutputCollector output, + public void map(ImmutableBytesWritable row, RowResult value, + OutputCollector output, @SuppressWarnings("unused") Reporter reporter) throws IOException { boolean content = false; - for (Map.Entry e: value.entrySet()) { + for (Map.Entry e: value.entrySet()) { Cell cell = e.getValue(); if (cell != null && cell.getValue().length > 0) { content = true; @@ -85,8 +85,8 @@ public class RowCounter extends TableMap implements Tool { sb.append(args[i]); } // Second argument is the table name. - TableMap.initJob(args[1], sb.toString(), this.getClass(), Text.class, - RowResult.class, c); + TableMap.initJob(args[1], sb.toString(), this.getClass(), + ImmutableBytesWritable.class, RowResult.class, c); c.setReducerClass(IdentityReducer.class); // First arg is the output directory. c.setOutputPath(new Path(args[0])); diff --git a/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java b/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java index b65216a2c3a..e33ac2df832 100644 --- a/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java +++ b/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java @@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobConfigurable; @@ -50,14 +51,13 @@ public class TableInputFormat extends TableInputFormatBase implements Path[] tableNames = job.getInputPaths(); String colArg = job.get(COLUMN_LIST); String[] colNames = colArg.split(" "); - Text[] m_cols = new Text[colNames.length]; + byte [][] m_cols = new byte[colNames.length][]; for (int i = 0; i < m_cols.length; i++) { - m_cols[i] = new Text(colNames[i]); + m_cols[i] = Bytes.toBytes(colNames[i]); } setInputColums(m_cols); try { - setHTable(new HTable(new HBaseConfiguration(job), new Text(tableNames[0] - .getName()))); + setHTable(new HTable(new HBaseConfiguration(job), tableNames[0].getName())); } catch (Exception e) { LOG.error(e); } @@ -66,7 +66,7 @@ public class TableInputFormat extends TableInputFormatBase implements /** {@inheritDoc} */ public void validateInput(JobConf job) throws IOException { // expecting exactly one path - Path[] tableNames = job.getInputPaths(); + Path [] tableNames = job.getInputPaths(); if (tableNames == null || tableNames.length > 1) { throw new IOException("expecting one table name"); } @@ -77,4 +77,4 @@ public class TableInputFormat extends TableInputFormatBase implements throw new IOException("expecting at least one column"); } } -} \ No newline at end of file +} diff --git a/src/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java b/src/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java index 1e283c487fb..b5f912f628e 100644 --- a/src/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java +++ b/src/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java @@ -1,14 +1,21 @@ -/* - * $Id$ +/** + * Copyright 2008 The Apache Software Foundation * - * Copyright Critical Software S.A., All Rights Reserved. - * (www.criticalsoftware.com) + * 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 * - * This software is the proprietary information of Critical Software S.A. - * Use is subject to license terms. + * http://www.apache.org/licenses/LICENSE-2.0 * - * Last changed on : $Date$ - * Last changed by : $Author$ + * 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.mapred; @@ -18,11 +25,13 @@ import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Scanner; import org.apache.hadoop.hbase.filter.RowFilterInterface; import org.apache.hadoop.hbase.filter.RowFilterSet; import org.apache.hadoop.hbase.filter.StopRowFilter; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.util.Writables; @@ -36,7 +45,7 @@ import org.apache.hadoop.mapred.Reporter; /** * A Base for {@link TableInputFormat}s. Receives a {@link HTable}, a * {@link Text}[] of input columns and optionally a {@link RowFilterInterface}. - * Subclasses may use other {@link TableRecordReader} implementations. + * Subclasses may use other TableRecordReader implementations. *

* An example of a subclass: * @@ -44,11 +53,11 @@ import org.apache.hadoop.mapred.Reporter; * * public void configure(JobConf job) { * HTable exampleTable = new HTable(new HBaseConfiguration(job), - * new Text("exampleTable")); + * Bytes.toBytes("exampleTable")); * // mandatory * setHTable(exampleTable); - * Text[] inputColumns = new Text[] { new Text("columnA"), - * new Text("columnB") }; + * Text[] inputColumns = new byte [][] { Bytes.toBytes("columnA"), + * Bytes.toBytes("columnB") }; * // mandatory * setInputColums(inputColumns); * RowFilterInterface exampleFilter = new RegExpRowFilter("keyPrefix.*"); @@ -62,9 +71,9 @@ import org.apache.hadoop.mapred.Reporter; * */ public abstract class TableInputFormatBase -implements InputFormat { +implements InputFormat { private final Log LOG = LogFactory.getLog(TableInputFormatBase.class); - private Text[] inputColumns; + private byte [][] inputColumns; private HTable table; private TableRecordReader tableRecordReader; private RowFilterInterface rowFilter; @@ -72,14 +81,14 @@ implements InputFormat { /** * Iterate over an HBase table data, return (Text, RowResult) pairs */ - protected class TableRecordReader implements RecordReader { - - private Text startRow; - private Text endRow; + protected class TableRecordReader + implements RecordReader { + private byte [] startRow; + private byte [] endRow; private RowFilterInterface trrRowFilter; private Scanner scanner; private HTable htable; - private Text[] trrInputColumns; + private byte [][] trrInputColumns; /** * Build the scanner. Not done in constructor to allow for extension. @@ -87,7 +96,7 @@ implements InputFormat { * @throws IOException */ public void init() throws IOException { - if ((endRow != null) && (endRow.getLength() > 0)) { + if ((endRow != null) && (endRow.length > 0)) { if (trrRowFilter != null) { final Set rowFiltersSet = new HashSet(); @@ -116,14 +125,14 @@ implements InputFormat { /** * @param inputColumns the columns to be placed in {@link RowResult}. */ - public void setInputColumns(Text[] inputColumns) { + public void setInputColumns(final byte [][] inputColumns) { this.trrInputColumns = inputColumns; } /** * @param startRow the first row in the split */ - public void setStartRow(Text startRow) { + public void setStartRow(final byte [] startRow) { this.startRow = startRow; } @@ -131,7 +140,7 @@ implements InputFormat { * * @param endRow the last row in the split */ - public void setEndRow(Text endRow) { + public void setEndRow(final byte [] endRow) { this.endRow = endRow; } @@ -148,12 +157,12 @@ implements InputFormat { } /** - * @return Text + * @return ImmutableBytesWritable * * @see org.apache.hadoop.mapred.RecordReader#createKey() */ - public Text createKey() { - return new Text(); + public ImmutableBytesWritable createKey() { + return new ImmutableBytesWritable(); } /** @@ -188,26 +197,26 @@ implements InputFormat { * @throws IOException */ @SuppressWarnings("unchecked") - public boolean next(Text key, RowResult value) throws IOException { + public boolean next(ImmutableBytesWritable key, RowResult value) + throws IOException { RowResult result = this.scanner.next(); - boolean hasMore = result != null; + boolean hasMore = result != null && result.size() > 0; if (hasMore) { - Writables.copyWritable(result.getRow(), key); + key.set(result.getRow()); Writables.copyWritable(result, value); } return hasMore; } - } /** - * Builds a {@link TableRecordReader}. If no {@link TableRecordReader} was - * provided uses the default. + * Builds a TableRecordReader. If no TableRecordReader was provided, uses + * the default. * * @see org.apache.hadoop.mapred.InputFormat#getRecordReader(InputSplit, * JobConf, Reporter) */ - public RecordReader getRecordReader(InputSplit split, + public RecordReader getRecordReader(InputSplit split, @SuppressWarnings("unused") JobConf job, @SuppressWarnings("unused") Reporter reporter) @@ -245,7 +254,7 @@ implements InputFormat { * @see org.apache.hadoop.mapred.InputFormat#getSplits(org.apache.hadoop.mapred.JobConf, int) */ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { - Text[] startKeys = this.table.getStartKeys(); + byte [][] startKeys = this.table.getStartKeys(); if (startKeys == null || startKeys.length == 0) { throw new IOException("Expecting at least one region"); } @@ -265,7 +274,7 @@ implements InputFormat { lastPos = startKeys.length % realNumSplits > i ? lastPos + 1 : lastPos; splits[i] = new TableSplit(this.table.getTableName(), startKeys[startPos], ((i + 1) < realNumSplits) ? startKeys[lastPos] - : new Text()); + : HConstants.EMPTY_START_ROW); if (LOG.isDebugEnabled()) { LOG.debug("split: " + i + "->" + splits[i]); } @@ -278,7 +287,7 @@ implements InputFormat { /** * @param inputColumns to be passed in {@link RowResult} to the map task. */ - protected void setInputColums(Text[] inputColumns) { + protected void setInputColums(byte [][] inputColumns) { this.inputColumns = inputColumns; } diff --git a/src/java/org/apache/hadoop/hbase/mapred/TableMap.java b/src/java/org/apache/hadoop/hbase/mapred/TableMap.java index befd51902a0..c67520f065c 100644 --- a/src/java/org/apache/hadoop/hbase/mapred/TableMap.java +++ b/src/java/org/apache/hadoop/hbase/mapred/TableMap.java @@ -22,9 +22,8 @@ package org.apache.hadoop.hbase.mapred; import java.io.IOException; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HStoreKey; -import org.apache.hadoop.io.MapWritable; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.mapred.JobConf; @@ -32,8 +31,6 @@ import org.apache.hadoop.mapred.MapReduceBase; import org.apache.hadoop.mapred.Mapper; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; -import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.hbase.io.BatchUpdate; /** * Scan an HBase table to sort by a specified sort column. @@ -44,7 +41,7 @@ import org.apache.hadoop.hbase.io.BatchUpdate; */ @SuppressWarnings("unchecked") public abstract class TableMap - extends MapReduceBase implements Mapper { + extends MapReduceBase implements Mapper { /** * Use this before submitting a TableMap job. It will * appropriately set up the JobConf. @@ -77,6 +74,6 @@ public abstract class TableMap * @param reporter * @throws IOException */ - public abstract void map(Text key, RowResult value, + public abstract void map(ImmutableBytesWritable key, RowResult value, OutputCollector output, Reporter reporter) throws IOException; } diff --git a/src/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java b/src/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java index 991ac164300..79ab5c658f7 100644 --- a/src/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java +++ b/src/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java @@ -27,7 +27,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.mapred.FileAlreadyExistsException; import org.apache.hadoop.mapred.InvalidJobConfException; import org.apache.hadoop.mapred.JobConf; @@ -39,7 +39,7 @@ import org.apache.hadoop.util.Progressable; /** * Convert Map/Reduce output and write it to an HBase table */ -public class TableOutputFormat extends OutputFormatBase { +public class TableOutputFormat extends OutputFormatBase { /** JobConf parameter that specifies the output table */ public static final String OUTPUT_TABLE = "hbase.mapred.outputtable"; @@ -50,7 +50,7 @@ public class TableOutputFormat extends OutputFormatBase { * and write to an HBase table */ protected class TableRecordWriter - implements RecordWriter { + implements RecordWriter { private HTable m_table; /** @@ -68,7 +68,7 @@ public class TableOutputFormat extends OutputFormatBase { } /** {@inheritDoc} */ - public void write(Text key, BatchUpdate value) throws IOException { + public void write(ImmutableBytesWritable key, BatchUpdate value) throws IOException { m_table.commit(value); } } @@ -84,7 +84,7 @@ public class TableOutputFormat extends OutputFormatBase { // expecting exactly one path - Text tableName = new Text(job.get(OUTPUT_TABLE)); + String tableName = job.get(OUTPUT_TABLE); HTable table = null; try { table = new HTable(new HBaseConfiguration(job), tableName); diff --git a/src/java/org/apache/hadoop/hbase/mapred/TableReduce.java b/src/java/org/apache/hadoop/hbase/mapred/TableReduce.java index 53a929049ed..126b9bb586b 100644 --- a/src/java/org/apache/hadoop/hbase/mapred/TableReduce.java +++ b/src/java/org/apache/hadoop/hbase/mapred/TableReduce.java @@ -22,8 +22,8 @@ package org.apache.hadoop.hbase.mapred; import java.io.IOException; import java.util.Iterator; -import org.apache.hadoop.io.MapWritable; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.mapred.JobConf; @@ -31,14 +31,13 @@ import org.apache.hadoop.mapred.MapReduceBase; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reducer; import org.apache.hadoop.mapred.Reporter; -import org.apache.hadoop.hbase.io.BatchUpdate; /** * Write a table, sorting by the input key */ @SuppressWarnings("unchecked") public abstract class TableReduce - extends MapReduceBase implements Reducer { + extends MapReduceBase implements Reducer { /** * Use this before submitting a TableReduce job. It will * appropriately set up the JobConf. @@ -52,7 +51,7 @@ public abstract class TableReduce values, - OutputCollector output, Reporter reporter) - throws IOException; -} + OutputCollector output, Reporter reporter) + throws IOException; +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/mapred/TableSplit.java b/src/java/org/apache/hadoop/hbase/mapred/TableSplit.java index a961f54dd06..a240f567f68 100644 --- a/src/java/org/apache/hadoop/hbase/mapred/TableSplit.java +++ b/src/java/org/apache/hadoop/hbase/mapred/TableSplit.java @@ -23,22 +23,22 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapred.InputSplit; /** * A table split corresponds to a key range [low, high) */ public class TableSplit implements InputSplit { - private Text m_tableName; - private Text m_startRow; - private Text m_endRow; + private byte [] m_tableName; + private byte [] m_startRow; + private byte [] m_endRow; /** default constructor */ public TableSplit() { - m_tableName = new Text(); - m_startRow = new Text(); - m_endRow = new Text(); + this(HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, + HConstants.EMPTY_BYTE_ARRAY); } /** @@ -47,25 +47,24 @@ public class TableSplit implements InputSplit { * @param startRow * @param endRow */ - public TableSplit(Text tableName, Text startRow, Text endRow) { - this(); - m_tableName.set(tableName); - m_startRow.set(startRow); - m_endRow.set(endRow); + public TableSplit(byte [] tableName, byte [] startRow, byte [] endRow) { + m_tableName = tableName; + m_startRow = startRow; + m_endRow = endRow; } /** @return table name */ - public Text getTableName() { + public byte [] getTableName() { return m_tableName; } /** @return starting row key */ - public Text getStartRow() { + public byte [] getStartRow() { return m_startRow; } /** @return end row key */ - public Text getEndRow() { + public byte [] getEndRow() { return m_endRow; } @@ -83,21 +82,22 @@ public class TableSplit implements InputSplit { /** {@inheritDoc} */ public void readFields(DataInput in) throws IOException { - m_tableName.readFields(in); - m_startRow.readFields(in); - m_endRow.readFields(in); + this.m_tableName = Bytes.readByteArray(in); + this.m_startRow = Bytes.readByteArray(in); + this.m_endRow = Bytes.readByteArray(in); } /** {@inheritDoc} */ public void write(DataOutput out) throws IOException { - m_tableName.write(out); - m_startRow.write(out); - m_endRow.write(out); + Bytes.writeByteArray(out, this.m_tableName); + Bytes.writeByteArray(out, this.m_startRow); + Bytes.writeByteArray(out, this.m_endRow); } /** {@inheritDoc} */ @Override public String toString() { - return m_tableName +"," + m_startRow + "," + m_endRow; + return Bytes.toString(m_tableName) +"," + Bytes.toString(m_startRow) + + "," + Bytes.toString(m_endRow); } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/master/AddColumn.java b/src/java/org/apache/hadoop/hbase/master/AddColumn.java index 5947fa93880..c46aa41e2fe 100644 --- a/src/java/org/apache/hadoop/hbase/master/AddColumn.java +++ b/src/java/org/apache/hadoop/hbase/master/AddColumn.java @@ -24,13 +24,12 @@ import java.io.IOException; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.io.Text; /** Instantiated to add a column family to a table */ class AddColumn extends ColumnOperation { private final HColumnDescriptor newColumn; - AddColumn(final HMaster master, final Text tableName, + AddColumn(final HMaster master, final byte [] tableName, final HColumnDescriptor newColumn) throws IOException { super(master, tableName); @@ -48,4 +47,4 @@ class AddColumn extends ColumnOperation { updateRegionInfo(server, m.getRegionName(), i); } } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/master/BaseScanner.java b/src/java/org/apache/hadoop/hbase/master/BaseScanner.java index 0dbfd8961a1..66a66d30f61 100644 --- a/src/java/org/apache/hadoop/hbase/master/BaseScanner.java +++ b/src/java/org/apache/hadoop/hbase/master/BaseScanner.java @@ -35,8 +35,8 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.Chore; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HServerInfo; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.UnknownScannerException; @@ -102,7 +102,7 @@ import org.apache.hadoop.hbase.ipc.HRegionInterface; abstract class BaseScanner extends Chore implements HConstants { static final Log LOG = LogFactory.getLog(BaseScanner.class.getName()); - protected final boolean rootRegion; + private final boolean rootRegion; protected final HMaster master; protected final RegionManager regionManager; @@ -153,26 +153,22 @@ abstract class BaseScanner extends Chore implements HConstants { // scan we go check if parents can be removed. Map splitParents = new HashMap(); - List emptyRows = new ArrayList(); + List emptyRows = new ArrayList(); try { regionServer = master.connection.getHRegionConnection(region.getServer()); - scannerId = - regionServer.openScanner(region.getRegionName(), COLUMN_FAMILY_ARRAY, - EMPTY_START_ROW, HConstants.LATEST_TIMESTAMP, null); - + scannerId = regionServer.openScanner(region.getRegionName(), + COLUMN_FAMILY_ARRAY, EMPTY_START_ROW, HConstants.LATEST_TIMESTAMP, null); int numberOfRegionsFound = 0; while (true) { RowResult values = regionServer.next(scannerId); if (values == null || values.size() == 0) { break; } - HRegionInfo info = master.getHRegionInfo(values.getRow(), values); if (info == null) { emptyRows.add(values.getRow()); continue; } - String serverName = Writables.cellToString(values.get(COL_SERVER)); long startCode = Writables.cellToLong(values.get(COL_STARTCODE)); if (LOG.isDebugEnabled()) { @@ -216,11 +212,9 @@ abstract class BaseScanner extends Chore implements HConstants { // Scan is finished. // First clean up any meta region rows which had null HRegionInfos - if (emptyRows.size() > 0) { - LOG.warn("Found " + emptyRows.size() + - " rows with empty HRegionInfo while scanning meta region " + - region.getRegionName()); + LOG.warn("Found " + emptyRows.size() + " rows with empty HRegionInfo " + + "while scanning meta region " + Bytes.toString(region.getRegionName())); master.deleteEmptyMetaRows(regionServer, region.getRegionName(), emptyRows); } @@ -262,7 +256,7 @@ abstract class BaseScanner extends Chore implements HConstants { * the filesystem. * @throws IOException */ - private boolean cleanupSplits(final Text metaRegionName, + private boolean cleanupSplits(final byte [] metaRegionName, final HRegionInterface srvr, final HRegionInfo parent, RowResult rowContent) throws IOException { @@ -302,9 +296,9 @@ abstract class BaseScanner extends Chore implements HConstants { * @return True if still has references to parent. * @throws IOException */ - private boolean hasReferences(final Text metaRegionName, - final HRegionInterface srvr, final Text parent, - RowResult rowContent, final Text splitColumn) + private boolean hasReferences(final byte [] metaRegionName, + final HRegionInterface srvr, final byte [] parent, + RowResult rowContent, final byte [] splitColumn) throws IOException { boolean result = false; HRegionInfo split = @@ -314,9 +308,9 @@ abstract class BaseScanner extends Chore implements HConstants { } Path tabledir = HTableDescriptor.getTableDir(master.rootdir, split.getTableDesc().getName()); - for (HColumnDescriptor family: split.getTableDesc().families().values()) { + for (HColumnDescriptor family: split.getTableDesc().getFamilies()) { Path p = HStoreFile.getMapDir(tabledir, split.getEncodedName(), - family.getFamilyName()); + family.getName()); // Look for reference files. Call listStatus with an anonymous // instance of PathFilter. @@ -390,8 +384,9 @@ abstract class BaseScanner extends Chore implements HConstants { && (storedInfo == null || storedInfo.getStartCode() != startCode)) { // The current assignment is invalid if (LOG.isDebugEnabled()) { - LOG.debug("Current assignment of " + info.getRegionName() + - " is not valid: storedInfo: " + storedInfo + ", startCode: " + + LOG.debug("Current assignment of " + + Bytes.toString(info.getRegionName()) + + " is not valid: serverInfo: " + storedInfo + ", passed startCode: " + startCode + ", storedInfo.startCode: " + ((storedInfo != null)? storedInfo.getStartCode(): -1) + ", unassignedRegions: " + diff --git a/src/java/org/apache/hadoop/hbase/master/ChangeTableState.java b/src/java/org/apache/hadoop/hbase/master/ChangeTableState.java index 51e008076bb..89866190e15 100644 --- a/src/java/org/apache/hadoop/hbase/master/ChangeTableState.java +++ b/src/java/org/apache/hadoop/hbase/master/ChangeTableState.java @@ -23,12 +23,13 @@ import java.io.IOException; import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import java.util.TreeMap; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.io.Text; /** Instantiated to enable or disable a table */ class ChangeTableState extends TableOperation { @@ -39,7 +40,7 @@ class ChangeTableState extends TableOperation { protected long lockid; - ChangeTableState(final HMaster master, final Text tableName, + ChangeTableState(final HMaster master, final byte [] tableName, final boolean onLine) throws IOException { super(master, tableName); @@ -118,9 +119,8 @@ class ChangeTableState extends TableOperation { // Cause regions being served to be taken off-line and disabled - HashMap localKillList = - new HashMap(); - + Map localKillList = + new TreeMap(Bytes.BYTES_COMPARATOR); for (HRegionInfo i: e.getValue()) { if (LOG.isDebugEnabled()) { LOG.debug("adding region " + i.getRegionName() + " to kill list"); @@ -130,7 +130,7 @@ class ChangeTableState extends TableOperation { // this marks the regions to be offlined once they are closed master.regionManager.markRegionForOffline(i.getRegionName()); } - Map killedRegions = + Map killedRegions = master.regionManager.removeMarkedToClose(serverName); if (killedRegions != null) { localKillList.putAll(killedRegions); diff --git a/src/java/org/apache/hadoop/hbase/master/ColumnOperation.java b/src/java/org/apache/hadoop/hbase/master/ColumnOperation.java index 3b992763d79..fcecaa4d379 100644 --- a/src/java/org/apache/hadoop/hbase/master/ColumnOperation.java +++ b/src/java/org/apache/hadoop/hbase/master/ColumnOperation.java @@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.master; import java.io.IOException; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.io.BatchUpdate; @@ -29,7 +28,7 @@ import org.apache.hadoop.hbase.util.Writables; abstract class ColumnOperation extends TableOperation { - protected ColumnOperation(final HMaster master, final Text tableName) + protected ColumnOperation(final HMaster master, final byte [] tableName) throws IOException { super(master, tableName); } @@ -44,7 +43,7 @@ abstract class ColumnOperation extends TableOperation { } } - protected void updateRegionInfo(HRegionInterface server, Text regionName, + protected void updateRegionInfo(HRegionInterface server, byte [] regionName, HRegionInfo i) throws IOException { BatchUpdate b = new BatchUpdate(i.getRegionName()); diff --git a/src/java/org/apache/hadoop/hbase/master/DeleteColumn.java b/src/java/org/apache/hadoop/hbase/master/DeleteColumn.java index 9e84fd01a58..a15d18aa565 100644 --- a/src/java/org/apache/hadoop/hbase/master/DeleteColumn.java +++ b/src/java/org/apache/hadoop/hbase/master/DeleteColumn.java @@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.master; import java.io.IOException; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.regionserver.HStoreFile; @@ -29,10 +28,10 @@ import org.apache.hadoop.hbase.ipc.HRegionInterface; /** Instantiated to remove a column family from a table */ class DeleteColumn extends ColumnOperation { - private final Text columnName; + private final byte [] columnName; - DeleteColumn(final HMaster master, final Text tableName, - final Text columnName) + DeleteColumn(final HMaster master, final byte [] tableName, + final byte [] columnName) throws IOException { super(master, tableName); this.columnName = columnName; @@ -43,16 +42,14 @@ class DeleteColumn extends ColumnOperation { throws IOException { Path tabledir = new Path(this.master.rootdir, tableName.toString()); for (HRegionInfo i: unservedRegions) { - i.getTableDesc().families().remove(columnName); + i.getTableDesc().removeFamily(columnName); updateRegionInfo(server, m.getRegionName(), i); - // Delete the directories used by the column - - String encodedName = i.getEncodedName(); + int encodedName = i.getEncodedName(); this.master.fs.delete( HStoreFile.getMapDir(tabledir, encodedName, columnName)); this.master.fs.delete( HStoreFile.getInfoDir(tabledir, encodedName, columnName)); } } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/master/HMaster.java b/src/java/org/apache/hadoop/hbase/master/HMaster.java index 928d19b420c..23b187369dd 100644 --- a/src/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/src/java/org/apache/hadoop/hbase/master/HMaster.java @@ -36,42 +36,40 @@ import org.apache.hadoop.dfs.DistributedFileSystem; import org.apache.hadoop.dfs.FSConstants; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HMsg; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HServerAddress; +import org.apache.hadoop.hbase.HServerInfo; +import org.apache.hadoop.hbase.HServerLoad; +import org.apache.hadoop.hbase.HStoreKey; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.LocalHBaseCluster; +import org.apache.hadoop.hbase.MasterNotRunningException; +import org.apache.hadoop.hbase.RemoteExceptionHandler; +import org.apache.hadoop.hbase.TableExistsException; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.ipc.HMasterInterface; +import org.apache.hadoop.hbase.ipc.HMasterRegionInterface; +import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.ipc.HbaseRPC; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.InfoServer; import org.apache.hadoop.hbase.util.Sleeper; import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.hbase.io.HbaseMapWritable; +import org.apache.hadoop.io.MapWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.Server; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HStoreKey; -import org.apache.hadoop.hbase.HServerAddress; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HServerLoad; -import org.apache.hadoop.hbase.RemoteExceptionHandler; -import org.apache.hadoop.hbase.HMsg; -import org.apache.hadoop.hbase.LocalHBaseCluster; -import org.apache.hadoop.hbase.HServerInfo; -import org.apache.hadoop.hbase.TableExistsException; -import org.apache.hadoop.hbase.MasterNotRunningException; - -import org.apache.hadoop.hbase.client.HConnection; -import org.apache.hadoop.hbase.client.HConnectionManager; -import org.apache.hadoop.hbase.client.HBaseAdmin; - -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.ipc.HMasterInterface; -import org.apache.hadoop.hbase.ipc.HMasterRegionInterface; -import org.apache.hadoop.hbase.regionserver.HRegion; - /** * HMaster is the "master server" for a HBase. * There is only one HMaster for a single HBase deployment. @@ -193,12 +191,12 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, this.conf.set(HConstants.HBASE_DIR, this.rootdir.toString()); this.rand = new Random(); Path rootRegionDir = - HRegion.getRegionDir(rootdir, HRegionInfo.rootRegionInfo); + HRegion.getRegionDir(rootdir, HRegionInfo.ROOT_REGIONINFO); LOG.info("Root region dir: " + rootRegionDir.toString()); try { // Make sure the root directory exists! - if(! fs.exists(rootdir)) { + if (!fs.exists(rootdir)) { fs.mkdirs(rootdir); FSUtils.setVersion(fs, rootdir); } else { @@ -206,11 +204,11 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, } if (!fs.exists(rootRegionDir)) { - LOG.info("bootstrap: creating ROOT and first META regions"); + LOG.info("BOOTSTRAP: creating ROOT and first META regions"); try { - HRegion root = HRegion.createHRegion(HRegionInfo.rootRegionInfo, - this.rootdir, this.conf); - HRegion meta = HRegion.createHRegion(HRegionInfo.firstMetaRegionInfo, + HRegion root = HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO, + this.rootdir, this.conf); + HRegion meta = HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO, this.rootdir, this.conf); // Add first region from the META table to the ROOT region. @@ -328,7 +326,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, /** * @return Read-only map of online regions. */ - public Map getOnlineMetaRegions() { + public Map getOnlineMetaRegions() { return regionManager.getOnlineMetaRegions(); } @@ -501,9 +499,6 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, if (LOG.isDebugEnabled()) { LOG.debug("Started service threads"); } - if (LOG.isDebugEnabled()) { - LOG.debug("Started service threads"); - } } /* @@ -526,11 +521,10 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, /** {@inheritDoc} */ @SuppressWarnings("unused") - public HbaseMapWritable regionServerStartup(HServerInfo serverInfo) + public MapWritable regionServerStartup(HServerInfo serverInfo) throws IOException { // register with server manager serverManager.regionServerStartup(serverInfo); - // send back some config info return createConfigurationSubset(); } @@ -539,12 +533,12 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, * @return Subset of configuration to pass initializing regionservers: e.g. * the filesystem to use and root directory to use. */ - protected HbaseMapWritable createConfigurationSubset() { - HbaseMapWritable mw = addConfig(new HbaseMapWritable(), HConstants.HBASE_DIR); + protected MapWritable createConfigurationSubset() { + MapWritable mw = addConfig(new MapWritable(), HConstants.HBASE_DIR); return addConfig(mw, "fs.default.name"); } - private HbaseMapWritable addConfig(final HbaseMapWritable mw, final String key) { + private MapWritable addConfig(final MapWritable mw, final String key) { mw.put(new Text(key), new Text(this.conf.get(key))); return mw; } @@ -588,7 +582,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, break; } createTable(newRegion); - LOG.info("created table " + desc.getName()); + LOG.info("created table " + desc.getNameAsString()); break; } catch (TableExistsException e) { throw e; @@ -603,14 +597,14 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, private synchronized void createTable(final HRegionInfo newRegion) throws IOException { - Text tableName = newRegion.getTableDesc().getName(); + byte [] tableName = newRegion.getTableDesc().getName(); // 1. Check to see if table already exists. Get meta region where // table would sit should it exist. Open scanner on it. If a region // for the table we want to create already exists, then table already // created. Throw already-exists exception. MetaRegion m = regionManager.getFirstMetaRegionForRegion(newRegion); - Text metaRegionName = m.getRegionName(); + byte [] metaRegionName = m.getRegionName(); HRegionInterface srvr = connection.getHRegionConnection(m.getServer()); long scannerid = srvr.openScanner(metaRegionName, COL_REGIONINFO_ARRAY, tableName, LATEST_TIMESTAMP, null); @@ -621,51 +615,50 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, // does not exist, scanner will return row after where our table would // be inserted if it exists so look for exact match on table name. if (data != null && data.size() > 0) { - if (HRegionInfo.getTableNameFromRegionName( - data.getRow()).equals(tableName)) { + byte [] tn = HRegionInfo.getTableNameFromRegionName(data.getRow()); + if (Bytes.equals(tn, tableName)) { // Then a region for this table already exists. Ergo table exists. - throw new TableExistsException(tableName.toString()); + throw new TableExistsException(Bytes.toString(tableName)); } } } finally { srvr.close(scannerid); } - regionManager.createRegion(newRegion, srvr, metaRegionName); } /** {@inheritDoc} */ - public void deleteTable(Text tableName) throws IOException { + public void deleteTable(final byte [] tableName) throws IOException { new TableDelete(this, tableName).process(); - LOG.info("deleted table: " + tableName); + LOG.info("deleted table: " + Bytes.toString(tableName)); } /** {@inheritDoc} */ - public void addColumn(Text tableName, HColumnDescriptor column) + public void addColumn(byte [] tableName, HColumnDescriptor column) throws IOException { new AddColumn(this, tableName, column).process(); } /** {@inheritDoc} */ - public void modifyColumn(Text tableName, Text columnName, + public void modifyColumn(byte [] tableName, byte [] columnName, HColumnDescriptor descriptor) throws IOException { new ModifyColumn(this, tableName, columnName, descriptor).process(); } /** {@inheritDoc} */ - public void deleteColumn(Text tableName, Text columnName) throws IOException { - new DeleteColumn(this, tableName, - HStoreKey.extractFamily(columnName)).process(); + public void deleteColumn(final byte [] tableName, final byte [] c) + throws IOException { + new DeleteColumn(this, tableName, HStoreKey.getFamily(c)).process(); } /** {@inheritDoc} */ - public void enableTable(Text tableName) throws IOException { + public void enableTable(final byte [] tableName) throws IOException { new ChangeTableState(this, tableName, true).process(); } /** {@inheritDoc} */ - public void disableTable(Text tableName) throws IOException { + public void disableTable(final byte [] tableName) throws IOException { new ChangeTableState(this, tableName, false).process(); } @@ -694,33 +687,41 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, * @return Null or found HRegionInfo. * @throws IOException */ - HRegionInfo getHRegionInfo(final Text row, final Map map) + HRegionInfo getHRegionInfo(final byte [] row, final Map map) throws IOException { Cell regioninfo = map.get(COL_REGIONINFO); if (regioninfo == null) { - LOG.warn(COL_REGIONINFO.toString() + " is empty for row: " + row + - "; has keys: " + map.keySet().toString()); + StringBuilder sb = new StringBuilder(); + for (byte [] e: map.keySet()) { + if (sb.length() > 0) { + sb.append(", "); + } + sb.append(Bytes.toString(e)); + } + LOG.warn(Bytes.toString(COL_REGIONINFO) + " is empty for row: " + + Bytes.toString(row) + "; has keys: " + sb.toString()); return null; } - return (HRegionInfo)Writables.getWritable(regioninfo.getValue(), new HRegionInfo()); + return Writables.getHRegionInfo(regioninfo.getValue()); } /* * When we find rows in a meta region that has an empty HRegionInfo, we * clean them up here. * - * @param server connection to server serving meta region + * @param s connection to server serving meta region * @param metaRegionName name of the meta region we scanned * @param emptyRows the row keys that had empty HRegionInfos */ - protected void deleteEmptyMetaRows(HRegionInterface server, - Text metaRegionName, - List emptyRows) { - for (Text regionName: emptyRows) { + protected void deleteEmptyMetaRows(HRegionInterface s, + byte [] metaRegionName, + List emptyRows) { + for (byte [] regionName: emptyRows) { try { - HRegion.removeRegionFromMETA(server, metaRegionName, regionName); - LOG.warn("Removed region: " + regionName + " from meta region: " + - metaRegionName + " because HRegionInfo was empty"); + HRegion.removeRegionFromMETA(s, metaRegionName, regionName); + LOG.warn("Removed region: " + Bytes.toString(regionName) + + " from meta region: " + + Bytes.toString(metaRegionName) + " because HRegionInfo was empty"); } catch (IOException e) { LOG.error("deleting region: " + regionName + " from meta region: " + metaRegionName, e); diff --git a/src/java/org/apache/hadoop/hbase/master/MetaRegion.java b/src/java/org/apache/hadoop/hbase/master/MetaRegion.java index ace2a7eaa39..d4e25ad1b70 100644 --- a/src/java/org/apache/hadoop/hbase/master/MetaRegion.java +++ b/src/java/org/apache/hadoop/hbase/master/MetaRegion.java @@ -19,43 +19,44 @@ */ package org.apache.hadoop.hbase.master; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HServerAddress; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.util.Bytes; /** Describes a meta region and its server */ public class MetaRegion implements Comparable { private final HServerAddress server; - private final Text regionName; - private final Text startKey; + private final byte [] regionName; + private final byte [] startKey; - MetaRegion(final HServerAddress server, final Text regionName, - final Text startKey) { + MetaRegion(final HServerAddress server, final byte [] regionName) { + this (server, regionName, HConstants.EMPTY_START_ROW); + } + + MetaRegion(final HServerAddress server, final byte [] regionName, + final byte [] startKey) { if (server == null) { throw new IllegalArgumentException("server cannot be null"); } this.server = server; - if (regionName == null) { throw new IllegalArgumentException("regionName cannot be null"); } - this.regionName = new Text(regionName); - - this.startKey = new Text(); - if (startKey != null) { - this.startKey.set(startKey); - } + this.regionName = regionName; + this.startKey = startKey; } /** {@inheritDoc} */ @Override public String toString() { - return "{regionname: " + this.regionName.toString() + ", startKey: <" + - this.startKey.toString() + ">, server: " + this.server.toString() + "}"; + return "{regionname: " + Bytes.toString(this.regionName) + + ", startKey: <" + Bytes.toString(this.startKey) + + ">, server: " + this.server.toString() + "}"; } /** @return the regionName */ - public Text getRegionName() { + public byte [] getRegionName() { return regionName; } @@ -65,7 +66,7 @@ public class MetaRegion implements Comparable { } /** @return the startKey */ - public Text getStartKey() { + public byte [] getStartKey() { return startKey; } @@ -87,9 +88,9 @@ public class MetaRegion implements Comparable { /** {@inheritDoc} */ public int compareTo(MetaRegion other) { - int result = this.regionName.compareTo(other.getRegionName()); + int result = Bytes.compareTo(this.regionName, other.getRegionName()); if(result == 0) { - result = this.startKey.compareTo(other.getStartKey()); + result = Bytes.compareTo(this.startKey, other.getStartKey()); if (result == 0) { // Might be on different host? result = this.server.compareTo(other.server); @@ -97,5 +98,4 @@ public class MetaRegion implements Comparable { } return result; } -} - +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/master/MetaScanner.java b/src/java/org/apache/hadoop/hbase/master/MetaScanner.java index bbf54ba356f..07e9b285039 100644 --- a/src/java/org/apache/hadoop/hbase/master/MetaScanner.java +++ b/src/java/org/apache/hadoop/hbase/master/MetaScanner.java @@ -97,7 +97,7 @@ class MetaScanner extends BaseScanner { MetaRegion region = null; while (!master.closed.get() && (region == null && metaRegionsToScan.size() > 0) && - !metaRegionsScanned()) { + !metaRegionsScanned()) { try { region = metaRegionsToScan.poll(master.threadWakeFrequency, TimeUnit.MILLISECONDS); @@ -146,11 +146,11 @@ class MetaScanner extends BaseScanner { */ synchronized boolean waitForMetaRegionsOrClose() { while (!master.closed.get()) { - if (regionManager.isInitialRootScanComplete() && - regionManager.numMetaRegions() == regionManager.numOnlineMetaRegions()) { + if (regionManager.isInitialRootScanComplete() && + regionManager.numMetaRegions() == + regionManager.numOnlineMetaRegions()) { break; } - try { wait(master.threadWakeFrequency); } catch (InterruptedException e) { @@ -163,7 +163,7 @@ class MetaScanner extends BaseScanner { /** * Add another meta region to scan to the queue. */ - void addMetaRegionToScan(MetaRegion m) throws InterruptedException { + void addMetaRegionToScan(MetaRegion m) { metaRegionsToScan.add(m); } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/master/ModifyColumn.java b/src/java/org/apache/hadoop/hbase/master/ModifyColumn.java index cce557b7bc2..c577354b37b 100644 --- a/src/java/org/apache/hadoop/hbase/master/ModifyColumn.java +++ b/src/java/org/apache/hadoop/hbase/master/ModifyColumn.java @@ -25,15 +25,14 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.InvalidColumnNameException; import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.io.Text; /** Instantiated to modify an existing column family on a table */ class ModifyColumn extends ColumnOperation { private final HColumnDescriptor descriptor; - private final Text columnName; + private final byte [] columnName; - ModifyColumn(final HMaster master, final Text tableName, - final Text columnName, HColumnDescriptor descriptor) + ModifyColumn(final HMaster master, final byte [] tableName, + final byte [] columnName, HColumnDescriptor descriptor) throws IOException { super(master, tableName); this.descriptor = descriptor; @@ -44,18 +43,13 @@ class ModifyColumn extends ColumnOperation { protected void postProcessMeta(MetaRegion m, HRegionInterface server) throws IOException { for (HRegionInfo i: unservedRegions) { - // get the column families map from the table descriptor - Map families = i.getTableDesc().families(); - - // if the table already has this column, then put the new descriptor - // version. - if (families.get(columnName) != null){ - families.put(columnName, descriptor); - updateRegionInfo(server, m.getRegionName(), i); - } else{ // otherwise, we have an error. + if (!i.getTableDesc().hasFamily(columnName)) { + i.getTableDesc().addFamily(descriptor); + updateRegionInfo(server, m.getRegionName(), i); + } else { // otherwise, we have an error. throw new InvalidColumnNameException("Column family '" + columnName + "' doesn't exist, so cannot be modified."); } } } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java b/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java index 486c04a6770..c7c52cb8971 100644 --- a/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java +++ b/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java @@ -21,11 +21,11 @@ package org.apache.hadoop.hbase.master; import java.io.IOException; +import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HServerAddress; import org.apache.hadoop.hbase.HServerInfo; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.util.Bytes; /** * ProcessRegionOpen is instantiated when a region server reports that it is @@ -42,12 +42,13 @@ class ProcessRegionOpen extends ProcessRegionStatusChange { * @param regionInfo * @throws IOException */ + @SuppressWarnings("unused") public ProcessRegionOpen(HMaster master, HServerInfo info, HRegionInfo regionInfo) throws IOException { super(master, regionInfo); this.serverAddress = info.getServerAddress(); - this.startCode = Writables.longToBytes(info.getStartCode()); + this.startCode = Bytes.toBytes(info.getStartCode()); } /** {@inheritDoc} */ @@ -61,8 +62,8 @@ class ProcessRegionOpen extends ProcessRegionStatusChange { Boolean result = new RetryableMetaOperation(this.metaRegion, this.master) { public Boolean call() throws IOException { - LOG.info(regionInfo.toString() + " open on " + serverAddress.toString()); - + LOG.info(regionInfo.getRegionNameAsString() + " open on " + + serverAddress.toString()); if (!metaRegionAvailable()) { // We can't proceed unless the meta region we are going to update // is online. metaRegionAvailable() has put this operation on the @@ -72,14 +73,12 @@ class ProcessRegionOpen extends ProcessRegionStatusChange { } // Register the newly-available Region's location. - - LOG.info("updating row " + regionInfo.getRegionName() + " in table " + - metaRegionName + " with startcode " + - Writables.bytesToLong(startCode) + " and server " + + LOG.info("updating row " + regionInfo.getRegionNameAsString() + + " in region " + Bytes.toString(metaRegionName) + + " with startcode " + Bytes.toLong(startCode) + " and server " + serverAddress.toString()); - BatchUpdate b = new BatchUpdate(regionInfo.getRegionName()); - b.put(COL_SERVER, Writables.stringToBytes(serverAddress.toString())); + b.put(COL_SERVER, Bytes.toBytes(serverAddress.toString())); b.put(COL_STARTCODE, startCode); server.batchUpdate(metaRegionName, b); if (isMetaTable) { diff --git a/src/java/org/apache/hadoop/hbase/master/ProcessRegionStatusChange.java b/src/java/org/apache/hadoop/hbase/master/ProcessRegionStatusChange.java index cf78c51af76..39b26340213 100644 --- a/src/java/org/apache/hadoop/hbase/master/ProcessRegionStatusChange.java +++ b/src/java/org/apache/hadoop/hbase/master/ProcessRegionStatusChange.java @@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.master; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.io.Text; /** * Abstract class that performs common operations for @@ -31,7 +30,7 @@ abstract class ProcessRegionStatusChange extends RegionServerOperation { protected final boolean isMetaTable; protected final HRegionInfo regionInfo; protected final MetaRegion metaRegion; - protected final Text metaRegionName; + protected final byte [] metaRegionName; /** * @param master @@ -42,7 +41,7 @@ abstract class ProcessRegionStatusChange extends RegionServerOperation { this.regionInfo = regionInfo; this.isMetaTable = regionInfo.isMetaTable(); if (isMetaTable) { - this.metaRegionName = HRegionInfo.rootRegionInfo.getRegionName(); + this.metaRegionName = HRegionInfo.ROOT_REGIONINFO.getRegionName(); this.metaRegion = new MetaRegion(master.getRootRegionLocation(), this.metaRegionName, HConstants.EMPTY_START_ROW); } else { @@ -61,7 +60,8 @@ abstract class ProcessRegionStatusChange extends RegionServerOperation { available = false; } } else { - if (!master.regionManager.isInitialRootScanComplete() || !metaTableAvailable()) { + if (!master.regionManager.isInitialRootScanComplete() || + !metaTableAvailable()) { // The root region has not been scanned or the meta table is not // available so we can't proceed. // Put the operation on the delayedToDoQueue @@ -71,4 +71,4 @@ abstract class ProcessRegionStatusChange extends RegionServerOperation { } return available; } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java b/src/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java index fd161b8db0d..e19a8afe089 100644 --- a/src/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java +++ b/src/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.regionserver.HLog; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.io.RowResult; /** @@ -53,10 +52,10 @@ class ProcessServerShutdown extends RegionServerOperation { private class ToDoEntry { boolean regionOffline; - Text row; - HRegionInfo info; + final byte [] row; + final HRegionInfo info; - ToDoEntry(Text row, HRegionInfo info) { + ToDoEntry(final byte [] row, final HRegionInfo info) { this.regionOffline = false; this.row = row; this.info = info; @@ -90,11 +89,11 @@ class ProcessServerShutdown extends RegionServerOperation { /** Finds regions that the dead region server was serving */ protected void scanMetaRegion(HRegionInterface server, long scannerId, - Text regionName) throws IOException { + byte [] regionName) throws IOException { List toDoList = new ArrayList(); Set regions = new HashSet(); - List emptyRows = new ArrayList(); + List emptyRows = new ArrayList(); try { while (true) { RowResult values = null; @@ -109,7 +108,7 @@ class ProcessServerShutdown extends RegionServerOperation { break; } - Text row = values.getRow(); + byte [] row = values.getRow(); if (LOG.isDebugEnabled() && row != null) { LOG.debug("shutdown scanner looking at " + row.toString()); @@ -118,13 +117,7 @@ class ProcessServerShutdown extends RegionServerOperation { // Check server name. If null, be conservative and treat as though // region had been on shutdown server (could be null because we // missed edits in hlog because hdfs does not do write-append). - String serverName; - try { - serverName = Writables.cellToString(values.get(COL_SERVER)); - } catch (UnsupportedEncodingException e) { - LOG.error("Server name", e); - break; - } + String serverName = Writables.cellToString(values.get(COL_SERVER)); if (serverName.length() > 0 && deadServerName.compareTo(serverName) != 0) { // This isn't the server you're looking for - move along @@ -205,10 +198,10 @@ class ProcessServerShutdown extends RegionServerOperation { master.getRootRegionLocation().getBindAddress()); } long scannerId = server.openScanner( - HRegionInfo.rootRegionInfo.getRegionName(), COLUMN_FAMILY_ARRAY, + HRegionInfo.ROOT_REGIONINFO.getRegionName(), COLUMN_FAMILY_ARRAY, EMPTY_START_ROW, HConstants.LATEST_TIMESTAMP, null); scanMetaRegion(server, scannerId, - HRegionInfo.rootRegionInfo.getRegionName()); + HRegionInfo.ROOT_REGIONINFO.getRegionName()); return true; } } @@ -268,7 +261,7 @@ class ProcessServerShutdown extends RegionServerOperation { // Scan the ROOT region Boolean result = new ScanRootRegion( new MetaRegion(master.getRootRegionLocation(), - HRegionInfo.rootRegionInfo.getRegionName(), + HRegionInfo.ROOT_REGIONINFO.getRegionName(), HConstants.EMPTY_START_ROW), this.master).doWithRetries(); if (result == null) { diff --git a/src/java/org/apache/hadoop/hbase/master/RegionManager.java b/src/java/org/apache/hadoop/hbase/master/RegionManager.java index 18952d8b12a..bcb76ccc034 100644 --- a/src/java/org/apache/hadoop/hbase/master/RegionManager.java +++ b/src/java/org/apache/hadoop/hbase/master/RegionManager.java @@ -24,17 +24,17 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; +import java.util.HashSet; import java.util.List; import java.util.ArrayList; -import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; import java.util.Collections; +import java.util.TreeSet; import java.util.concurrent.ConcurrentHashMap; -import org.apache.hadoop.io.Text; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HConstants; @@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.HMsg; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.util.Writables; @@ -67,8 +68,9 @@ class RegionManager implements HConstants { private final AtomicInteger numberOfMetaRegions = new AtomicInteger(); /** These are the online meta regions */ - private final SortedMap onlineMetaRegions = - Collections.synchronizedSortedMap(new TreeMap()); + private final SortedMap onlineMetaRegions = + Collections.synchronizedSortedMap(new TreeMap(Bytes.BYTES_COMPARATOR)); /** * The 'unassignedRegions' table maps from a HRegionInfo to a timestamp that @@ -90,25 +92,25 @@ class RegionManager implements HConstants { * Regions that have been assigned, and the server has reported that it has * started serving it, but that we have not yet recorded in the meta table. */ - private final Set pendingRegions = - Collections.synchronizedSet(new HashSet()); + private final Set pendingRegions = + Collections.synchronizedSet(new TreeSet(Bytes.BYTES_COMPARATOR)); /** * List of regions that are going to be closed. */ - private final Map> regionsToClose = - new ConcurrentHashMap>(); + private final Map> regionsToClose = + new ConcurrentHashMap>(); /** Regions that are in the process of being closed */ - private final Set closingRegions = - Collections.synchronizedSet(new HashSet()); + private final Set closingRegions = + Collections.synchronizedSet(new TreeSet(Bytes.BYTES_COMPARATOR)); /** * Set of regions that, once closed, should be marked as offline so that they * are not reassigned. */ - private final Set regionsToOffline = - Collections.synchronizedSet(new HashSet()); + private final Set regionsToOffline = + Collections.synchronizedSet(new TreeSet(Bytes.BYTES_COMPARATOR)); // How many regions to assign a server at a time. private final int maxAssignInOneGo; @@ -147,7 +149,7 @@ class RegionManager implements HConstants { void unassignRootRegion() { rootRegionLocation.set(null); if (!master.shutdownRequested) { - unassignedRegions.put(HRegionInfo.rootRegionInfo, ZERO_L); + unassignedRegions.put(HRegionInfo.ROOT_REGIONINFO, ZERO_L); } } @@ -161,9 +163,7 @@ class RegionManager implements HConstants { */ void assignRegions(HServerInfo info, String serverName, HRegionInfo[] mostLoadedRegions, ArrayList returnMsgs) { - HServerLoad thisServersLoad = info.getLoad(); - synchronized (unassignedRegions) { // We need to hold a lock on assign attempts while we figure out what to // do so that multiple threads do not execute this method in parallel @@ -172,7 +172,6 @@ class RegionManager implements HConstants { // figure out what regions need to be assigned and aren't currently being // worked on elsewhere. Set regionsToAssign = regionsAwaitingAssignment(); - if (regionsToAssign.size() == 0) { // There are no regions waiting to be assigned. This is an opportunity // for us to check if this server is overloaded. @@ -252,8 +251,9 @@ class RegionManager implements HConstants { long now = System.currentTimeMillis(); for (HRegionInfo regionInfo: regionsToAssign) { - LOG.info("assigning region " + regionInfo.getRegionName() + - " to server " + serverName); + LOG.info("assigning region " + + Bytes.toString(regionInfo.getRegionName())+ + " to server " + serverName); unassignedRegions.put(regionInfo, Long.valueOf(now)); returnMsgs.add(new HMsg(HMsg.MSG_REGION_OPEN, regionInfo)); if (--nregions <= 0) { @@ -376,7 +376,8 @@ class RegionManager implements HConstants { final String serverName, final ArrayList returnMsgs) { long now = System.currentTimeMillis(); for (HRegionInfo regionInfo: regionsToAssign) { - LOG.info("assigning region " + regionInfo.getRegionName() + + LOG.info("assigning region " + + Bytes.toString(regionInfo.getRegionName()) + " to the only server " + serverName); unassignedRegions.put(regionInfo, Long.valueOf(now)); returnMsgs.add(new HMsg(HMsg.MSG_REGION_OPEN, regionInfo)); @@ -428,9 +429,9 @@ class RegionManager implements HConstants { /** * @return Read-only map of online regions. */ - public Map getOnlineMetaRegions() { + public Map getOnlineMetaRegions() { synchronized (onlineMetaRegions) { - return new TreeMap(onlineMetaRegions); + return Collections.unmodifiableMap(onlineMetaRegions); } } @@ -505,8 +506,8 @@ class RegionManager implements HConstants { * @param tableName Table you need to know all the meta regions for * @return set of MetaRegion objects that contain the table */ - public Set getMetaRegionsForTable(Text tableName) { - Text firstMetaRegion = null; + public Set getMetaRegionsForTable(byte [] tableName) { + byte [] firstMetaRegion = null; Set metaRegions = new HashSet(); synchronized (onlineMetaRegions) { @@ -533,7 +534,7 @@ class RegionManager implements HConstants { * @throws IOException */ public void createRegion(HRegionInfo newRegion, HRegionInterface server, - Text metaRegionName) + byte [] metaRegionName) throws IOException { // 2. Create the HRegion HRegion region = @@ -541,7 +542,7 @@ class RegionManager implements HConstants { // 3. Insert into meta HRegionInfo info = region.getRegionInfo(); - Text regionName = region.getRegionName(); + byte [] regionName = region.getRegionName(); BatchUpdate b = new BatchUpdate(regionName); b.put(COL_REGIONINFO, Writables.getBytes(info)); server.batchUpdate(metaRegionName, b); @@ -587,7 +588,7 @@ class RegionManager implements HConstants { * @param startKey name of the meta region to check * @return true if the region is online, false otherwise */ - public boolean isMetaRegionOnline(Text startKey) { + public boolean isMetaRegionOnline(byte [] startKey) { return onlineMetaRegions.containsKey(startKey); } @@ -595,7 +596,7 @@ class RegionManager implements HConstants { * Set an online MetaRegion offline - remove it from the map. * @param startKey region name */ - public void offlineMetaRegion(Text startKey) { + public void offlineMetaRegion(byte [] startKey) { onlineMetaRegions.remove(startKey); } @@ -615,7 +616,7 @@ class RegionManager implements HConstants { * @param regionName name of the region * @return true if pending, false otherwise */ - public boolean isPending(Text regionName) { + public boolean isPending(byte [] regionName) { return pendingRegions.contains(regionName); } @@ -636,7 +637,7 @@ class RegionManager implements HConstants { * Set a region to pending assignment * @param regionName */ - public void setPending(Text regionName) { + public void setPending(byte [] regionName) { pendingRegions.add(regionName); } @@ -644,7 +645,7 @@ class RegionManager implements HConstants { * Unset region's pending status * @param regionName */ - public void noLongerPending(Text regionName) { + public void noLongerPending(byte [] regionName) { pendingRegions.remove(regionName); } @@ -679,7 +680,7 @@ class RegionManager implements HConstants { */ public void markToClose(String serverName, HRegionInfo info) { synchronized (regionsToClose) { - Map serverToClose = regionsToClose.get(serverName); + Map serverToClose = regionsToClose.get(serverName); if (serverToClose != null) { serverToClose.put(info.getRegionName(), info); } @@ -691,10 +692,10 @@ class RegionManager implements HConstants { * @param serverName address info of server * @param map map of region names to region infos of regions to close */ - public void markToCloseBulk(String serverName, - Map map) { + public void markToCloseBulk(String serverName, + Map map) { synchronized (regionsToClose) { - Map regions = regionsToClose.get(serverName); + Map regions = regionsToClose.get(serverName); if (regions != null) { regions.putAll(map); } else { @@ -711,7 +712,7 @@ class RegionManager implements HConstants { * @param serverName * @return map of region names to region infos to close */ - public Map removeMarkedToClose(String serverName) { + public Map removeMarkedToClose(String serverName) { return regionsToClose.remove(serverName); } @@ -721,9 +722,9 @@ class RegionManager implements HConstants { * @param regionName name of the region we might want to close * @return true if the region is marked to close, false otherwise */ - public boolean isMarkedToClose(String serverName, Text regionName) { + public boolean isMarkedToClose(String serverName, byte [] regionName) { synchronized (regionsToClose) { - Map serverToClose = regionsToClose.get(serverName); + Map serverToClose = regionsToClose.get(serverName); return (serverToClose != null && serverToClose.containsKey(regionName)); } } @@ -734,9 +735,9 @@ class RegionManager implements HConstants { * @param serverName address info of server * @param regionName name of the region */ - public void noLongerMarkedToClose(String serverName, Text regionName) { + public void noLongerMarkedToClose(String serverName, byte [] regionName) { synchronized (regionsToClose) { - Map serverToClose = regionsToClose.get(serverName); + Map serverToClose = regionsToClose.get(serverName); if (serverToClose != null) { serverToClose.remove(regionName); } @@ -757,7 +758,7 @@ class RegionManager implements HConstants { * @param regionName * @return true if the region is marked as closing, false otherwise */ - public boolean isClosing(Text regionName) { + public boolean isClosing(byte [] regionName) { return closingRegions.contains(regionName); } @@ -765,7 +766,7 @@ class RegionManager implements HConstants { * Set a region as no longer closing (closed?) * @param regionName */ - public void noLongerClosing(Text regionName) { + public void noLongerClosing(byte [] regionName) { closingRegions.remove(regionName); } @@ -773,7 +774,7 @@ class RegionManager implements HConstants { * Mark a region as closing * @param regionName */ - public void setClosing(Text regionName) { + public void setClosing(byte [] regionName) { closingRegions.add(regionName); } @@ -790,7 +791,7 @@ class RegionManager implements HConstants { * Note that a region should be offlined as soon as its closed. * @param regionName */ - public void markRegionForOffline(Text regionName) { + public void markRegionForOffline(byte [] regionName) { regionsToOffline.add(regionName); } @@ -799,7 +800,7 @@ class RegionManager implements HConstants { * @param regionName * @return true if marked for offline, false otherwise */ - public boolean isMarkedForOffline(Text regionName) { + public boolean isMarkedForOffline(byte [] regionName) { return regionsToOffline.contains(regionName); } @@ -807,7 +808,7 @@ class RegionManager implements HConstants { * Region was offlined as planned, remove it from the list to offline * @param regionName */ - public void regionOfflined(Text regionName) { + public void regionOfflined(byte [] regionName) { regionsToOffline.remove(regionName); } diff --git a/src/java/org/apache/hadoop/hbase/master/RootScanner.java b/src/java/org/apache/hadoop/hbase/master/RootScanner.java index 6e730d99baa..6dfa19315d4 100644 --- a/src/java/org/apache/hadoop/hbase/master/RootScanner.java +++ b/src/java/org/apache/hadoop/hbase/master/RootScanner.java @@ -44,8 +44,10 @@ class RootScanner extends BaseScanner { try { // Don't interrupt us while we're working synchronized(scannerLock) { - scanRegion(new MetaRegion(master.getRootRegionLocation(), - HRegionInfo.rootRegionInfo.getRegionName(), null)); + if (master.getRootRegionLocation() != null) { + scanRegion(new MetaRegion(master.getRootRegionLocation(), + HRegionInfo.ROOT_REGIONINFO.getRegionName())); + } } scanSuccessful = true; } catch (IOException e) { @@ -71,4 +73,4 @@ class RootScanner extends BaseScanner { protected void maintenanceScan() { scanRoot(); } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/master/RowMap.java b/src/java/org/apache/hadoop/hbase/master/RowMap.java deleted file mode 100644 index 476af33e5ed..00000000000 --- a/src/java/org/apache/hadoop/hbase/master/RowMap.java +++ /dev/null @@ -1,85 +0,0 @@ -/** - * Copyright 2008 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.master; - -import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import org.apache.hadoop.io.Text; -import org.apache.hadoop.hbase.HStoreKey; -import org.apache.hadoop.hbase.io.HbaseMapWritable; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; - -/* - * Data structure used to return results out of the toRowMap method. - */ -class RowMap { - private static final Log LOG = LogFactory.getLog(RowMap.class.getName()); - - private final Text row; - private final SortedMap map; - - RowMap(final Text r, final SortedMap m) { - this.row = r; - this.map = m; - } - - Text getRow() { - return this.row; - } - - SortedMap getMap() { - return this.map; - } - - /* - * Convert an HbaseMapWritable to a Map keyed by column. - * Utility method used scanning meta regions - * @param mw The MapWritable to convert. Cannot be null. - * @return Returns a SortedMap currently. TODO: This looks like it could - * be a plain Map. - */ - static RowMap fromHbaseMapWritable(HbaseMapWritable mw) { - if (mw == null) { - throw new IllegalArgumentException("Passed MapWritable cannot be null"); - } - SortedMap m = new TreeMap(); - Text row = null; - for (Map.Entry e: mw.entrySet()) { - HStoreKey key = (HStoreKey) e.getKey(); - Text thisRow = key.getRow(); - if (row == null) { - row = thisRow; - } else { - if (!row.equals(thisRow)) { - LOG.error("Multiple rows in same scanner result set. firstRow=" + - row + ", currentRow=" + thisRow); - } - } - m.put(key.getColumn(), ((ImmutableBytesWritable) e.getValue()).get()); - } - return new RowMap(row, m); - } -} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/master/ServerManager.java b/src/java/org/apache/hadoop/hbase/master/ServerManager.java index 80d5de2fef6..19cbb8cb876 100644 --- a/src/java/org/apache/hadoop/hbase/master/ServerManager.java +++ b/src/java/org/apache/hadoop/hbase/master/ServerManager.java @@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.Leases; import org.apache.hadoop.hbase.LeaseListener; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.io.Text; /** * The ServerManager class manages info about region servers - HServerInfo, @@ -153,11 +152,10 @@ class ServerManager implements HConstants { * * @throws IOException */ - public HMsg[] regionServerReport(HServerInfo serverInfo, HMsg msgs[], + public HMsg [] regionServerReport(HServerInfo serverInfo, HMsg msgs[], HRegionInfo[] mostLoadedRegions) throws IOException { String serverName = serverInfo.getServerAddress().toString().trim(); - if (msgs.length > 0) { if (msgs[0].getMsg() == HMsg.MSG_REPORT_EXITING) { processRegionServerExit(serverName, msgs); @@ -183,7 +181,7 @@ class ServerManager implements HConstants { return new HMsg[0]; } // Tell the server to stop serving any user regions - return new HMsg[]{new HMsg(HMsg.MSG_REGIONSERVER_QUIESCE)}; + return new HMsg [] {new HMsg(HMsg.MSG_REGIONSERVER_QUIESCE)}; } } @@ -191,7 +189,7 @@ class ServerManager implements HConstants { // Tell server to shut down if we are shutting down. This should // happen after check of MSG_REPORT_EXITING above, since region server // will send us one of these messages after it gets MSG_REGIONSERVER_STOP - return new HMsg[]{new HMsg(HMsg.MSG_REGIONSERVER_STOP)}; + return new HMsg [] {new HMsg(HMsg.MSG_REGIONSERVER_STOP)}; } HServerInfo storedInfo = serversToServerInfo.get(serverName); @@ -325,14 +323,13 @@ class ServerManager implements HConstants { HRegionInfo[] mostLoadedRegions, HMsg incomingMsgs[]) throws IOException { ArrayList returnMsgs = new ArrayList(); - Map regionsToKill = + Map regionsToKill = master.regionManager.removeMarkedToClose(serverName); // Get reports on what the RegionServer did. for (int i = 0; i < incomingMsgs.length; i++) { if (LOG.isDebugEnabled()) { - LOG.debug("Received " + incomingMsgs[i].toString() + " from " + - serverName); + LOG.debug("Received " + incomingMsgs[i] + " from " + serverName); } HRegionInfo region = incomingMsgs[i].getRegionInfo(); @@ -369,7 +366,6 @@ class ServerManager implements HConstants { master.regionManager.setClosing(i.getRegionName()); } } - // Figure out what the RegionServer ought to do, and write back. master.regionManager.assignRegions(serverInfo, serverName, mostLoadedRegions, returnMsgs); @@ -410,7 +406,6 @@ class ServerManager implements HConstants { HRegionInfo region, ArrayList returnMsgs) throws IOException { boolean duplicateAssignment = false; - if (!master.regionManager.isUnassigned(region)) { if (region.isRootRegion()) { // Root region @@ -448,9 +443,6 @@ class ServerManager implements HConstants { // and then try to reopen it elsewhere; that's not what we want. returnMsgs.add(new HMsg(HMsg.MSG_REGION_CLOSE_WITHOUT_REPORT, region)); } else { - LOG.info(serverInfo.getServerAddress().toString() + " serving " + - region.getRegionName()); - // it was assigned, and it's not a duplicate assignment, so take it out // of the unassigned list. master.regionManager.noLongerUnassigned(region); @@ -462,7 +454,6 @@ class ServerManager implements HConstants { // Note that the table has been assigned and is waiting for the // meta table to be updated. master.regionManager.setPending(region.getRegionName()); - // Queue up an update to note the region location. try { master.toDoQueue.put( @@ -658,7 +649,9 @@ class ServerManager implements HConstants { serversToServerInfo.values()); try { serversToServerInfo.wait(master.threadWakeFrequency); - } catch (InterruptedException e) {} + } catch (InterruptedException e) { + // continue + } } } } diff --git a/src/java/org/apache/hadoop/hbase/master/TableDelete.java b/src/java/org/apache/hadoop/hbase/master/TableDelete.java index 404c3bba7ed..4263f2bb780 100644 --- a/src/java/org/apache/hadoop/hbase/master/TableDelete.java +++ b/src/java/org/apache/hadoop/hbase/master/TableDelete.java @@ -26,7 +26,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.TableNotDisabledException; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.ipc.HRegionInterface; @@ -36,7 +35,7 @@ import org.apache.hadoop.hbase.ipc.HRegionInterface; */ class TableDelete extends TableOperation { - TableDelete(final HMaster master, final Text tableName) throws IOException { + TableDelete(final HMaster master, final byte [] tableName) throws IOException { super(master, tableName); } diff --git a/src/java/org/apache/hadoop/hbase/master/TableOperation.java b/src/java/org/apache/hadoop/hbase/master/TableOperation.java index 8f68cf0c831..9fbaf7281ad 100644 --- a/src/java/org/apache/hadoop/hbase/master/TableOperation.java +++ b/src/java/org/apache/hadoop/hbase/master/TableOperation.java @@ -33,9 +33,9 @@ import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.HServerInfo; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.RemoteExceptionHandler; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.util.Sleeper; @@ -50,13 +50,13 @@ abstract class TableOperation implements HConstants { protected static final Log LOG = LogFactory.getLog(TableOperation.class); protected Set metaRegions; - protected Text tableName; + protected byte [] tableName; protected Set unservedRegions; protected HMaster master; protected final int numRetries; protected final Sleeper sleeper; - protected TableOperation(final HMaster master, final Text tableName) + protected TableOperation(final HMaster master, final byte [] tableName) throws IOException { this.sleeper = master.sleeper; this.numRetries = master.numRetries; @@ -94,7 +94,7 @@ abstract class TableOperation implements HConstants { long scannerId = server.openScanner(m.getRegionName(), COLUMN_FAMILY_ARRAY, tableName, HConstants.LATEST_TIMESTAMP, null); - List emptyRows = new ArrayList(); + List emptyRows = new ArrayList(); try { while (true) { RowResult values = server.next(scannerId); @@ -109,7 +109,7 @@ abstract class TableOperation implements HConstants { } String serverName = Writables.cellToString(values.get(COL_SERVER)); long startCode = Writables.cellToLong(values.get(COL_STARTCODE)); - if (info.getTableDesc().getName().compareTo(tableName) > 0) { + if (Bytes.compareTo(info.getTableDesc().getName(), tableName) > 0) { break; // Beyond any more entries for this table } diff --git a/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java b/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java index 8d81231f498..1bd01ff71a6 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java @@ -26,7 +26,6 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantLock; -import org.apache.hadoop.io.Text; import org.apache.hadoop.util.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -36,6 +35,7 @@ import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; /** @@ -83,7 +83,7 @@ implements RegionUnavailableListener, HConstants { lock.lock(); try { // Don't interrupt us while we are working - Text midKey = r.compactStores(); + byte [] midKey = r.compactStores(); if (midKey != null) { split(r, midKey); } @@ -119,7 +119,8 @@ implements RegionUnavailableListener, HConstants { * @param r HRegion store belongs to */ public synchronized void compactionRequested(HRegion r) { - LOG.debug("Compaction requested for region: " + r.getRegionName()); + LOG.debug("Compaction requested for region: " + + Bytes.toString(r.getRegionName())); synchronized (regionsInQueue) { if (!regionsInQueue.contains(r)) { compactionQueue.add(r); @@ -128,7 +129,7 @@ implements RegionUnavailableListener, HConstants { } } - private void split(final HRegion region, final Text midKey) + private void split(final HRegion region, final byte [] midKey) throws IOException { final HRegionInfo oldRegionInfo = region.getRegionInfo(); final HRegion[] newRegions = region.splitRegion(this, midKey); @@ -190,11 +191,13 @@ implements RegionUnavailableListener, HConstants { } /** {@inheritDoc} */ - public void closing(@SuppressWarnings("unused") final Text regionName) { + public void closing(@SuppressWarnings("unused") final byte [] regionName) { + // continue } /** {@inheritDoc} */ - public void closed(@SuppressWarnings("unused") final Text regionName) { + public void closed(@SuppressWarnings("unused") final byte [] regionName) { + // continue } /** @@ -205,4 +208,4 @@ implements RegionUnavailableListener, HConstants { this.interrupt(); } } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HAbstractScanner.java b/src/java/org/apache/hadoop/hbase/regionserver/HAbstractScanner.java index 5f5bd192c17..1ec45bfd96d 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HAbstractScanner.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HAbstractScanner.java @@ -20,9 +20,10 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; +import java.util.HashMap; import java.util.Iterator; +import java.util.Map; import java.util.SortedMap; -import java.util.TreeMap; import java.util.Vector; import java.util.Map.Entry; import java.util.regex.Pattern; @@ -30,7 +31,7 @@ import java.util.regex.Pattern; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HStoreKey; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.util.Bytes; /** * Abstract base class that implements the InternalScanner. @@ -64,20 +65,22 @@ public abstract class HAbstractScanner implements InternalScanner { private static class ColumnMatcher { private boolean wildCardmatch; private MATCH_TYPE matchType; - private Text family; + private byte [] family; private Pattern columnMatcher; - private Text col; + private byte [] col; - ColumnMatcher(final Text col) throws IOException { - Text qualifier = HStoreKey.extractQualifier(col); + ColumnMatcher(final byte [] col) throws IOException { + byte [][] parse = HStoreKey.parseColumn(col); + // First position has family. Second has qualifier. + byte [] qualifier = parse[1]; try { - if (qualifier == null || qualifier.getLength() == 0) { + if (qualifier == null || qualifier.length == 0) { this.matchType = MATCH_TYPE.FAMILY_ONLY; - this.family = HStoreKey.extractFamily(col).toText(); + this.family = parse[0]; this.wildCardmatch = true; - } else if(isRegexPattern.matcher(qualifier.toString()).matches()) { + } else if (isRegexPattern.matcher(Bytes.toString(qualifier)).matches()) { this.matchType = MATCH_TYPE.REGEX; - this.columnMatcher = Pattern.compile(col.toString()); + this.columnMatcher = Pattern.compile(Bytes.toString(col)); this.wildCardmatch = true; } else { this.matchType = MATCH_TYPE.SIMPLE; @@ -85,18 +88,19 @@ public abstract class HAbstractScanner implements InternalScanner { this.wildCardmatch = false; } } catch(Exception e) { - throw new IOException("Column: " + col + ": " + e.getMessage()); + throw new IOException("Column: " + Bytes.toString(col) + ": " + + e.getMessage()); } } /** Matching method */ - boolean matches(Text c) throws IOException { + boolean matches(final byte [] c) throws IOException { if(this.matchType == MATCH_TYPE.SIMPLE) { - return c.equals(this.col); + return Bytes.equals(c, this.col); } else if(this.matchType == MATCH_TYPE.FAMILY_ONLY) { - return HStoreKey.extractFamily(c).equals(this.family); - } else if(this.matchType == MATCH_TYPE.REGEX) { - return this.columnMatcher.matcher(c.toString()).matches(); + return HStoreKey.matchingFamily(this.family, c); + } else if (this.matchType == MATCH_TYPE.REGEX) { + return this.columnMatcher.matcher(Bytes.toString(c)).matches(); } else { throw new IOException("Invalid match type: " + this.matchType); } @@ -107,8 +111,10 @@ public abstract class HAbstractScanner implements InternalScanner { } } - // Holds matchers for each column family - protected TreeMap> okCols; + // Holds matchers for each column family. Its keyed by the byte [] hashcode + // which you can get by calling Bytes.mapKey. + private Map> okCols = + new HashMap>(); // True when scanning is done protected volatile boolean scannerClosed = false; @@ -120,14 +126,13 @@ public abstract class HAbstractScanner implements InternalScanner { private boolean multipleMatchers; /** Constructor for abstract base class */ - HAbstractScanner(long timestamp, Text[] targetCols) throws IOException { + HAbstractScanner(long timestamp, byte [][] targetCols) throws IOException { this.timestamp = timestamp; this.wildcardMatch = false; this.multipleMatchers = false; - this.okCols = new TreeMap>(); for(int i = 0; i < targetCols.length; i++) { - Text family = HStoreKey.extractFamily(targetCols[i]).toText(); - Vector matchers = okCols.get(family); + Integer key = HStoreKey.getFamilyMapKey(targetCols[i]); + Vector matchers = okCols.get(key); if (matchers == null) { matchers = new Vector(); } @@ -139,7 +144,7 @@ public abstract class HAbstractScanner implements InternalScanner { if (matchers.size() > 1) { this.multipleMatchers = true; } - okCols.put(family, matchers); + okCols.put(key, matchers); } } @@ -154,14 +159,14 @@ public abstract class HAbstractScanner implements InternalScanner { * * @throws IOException */ - protected boolean columnMatch(final Text column) throws IOException { + protected boolean columnMatch(final byte [] column) throws IOException { Vector matchers = - this.okCols.get(HStoreKey.extractFamily(column)); + this.okCols.get(HStoreKey.getFamilyMapKey(column)); if (matchers == null) { return false; } for(int m = 0; m < matchers.size(); m++) { - if(matchers.get(m).matches(column)) { + if (matchers.get(m).matches(column)) { return true; } } @@ -178,10 +183,11 @@ public abstract class HAbstractScanner implements InternalScanner { return this.multipleMatchers; } - public abstract boolean next(HStoreKey key, SortedMap results) + public abstract boolean next(HStoreKey key, + SortedMap results) throws IOException; - public Iterator>> iterator() { + public Iterator>> iterator() { throw new UnsupportedOperationException("Unimplemented serverside. " + "next(HStoreKey, StortedMap(...) is more efficient"); } diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HLog.java b/src/java/org/apache/hadoop/hbase/regionserver/HLog.java index 7e00411db70..ad4047caaf0 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HLog.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HLog.java @@ -19,16 +19,14 @@ */ package org.apache.hadoop.hbase.regionserver; -import java.io.FileNotFoundException; import java.io.EOFException; +import java.io.FileNotFoundException; import java.io.IOException; import java.util.Collections; -import java.util.HashMap; import java.util.Map; import java.util.SortedMap; import java.util.TreeMap; import java.util.TreeSet; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -39,19 +37,19 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HStoreKey; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.RemoteExceptionHandler; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.SequenceFile.Reader; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HStoreKey; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.RemoteExceptionHandler; -import org.apache.hadoop.hbase.util.FSUtils; - /** * HLog stores all the edits to the HStore. * @@ -97,8 +95,8 @@ import org.apache.hadoop.hbase.util.FSUtils; public class HLog implements HConstants { private static final Log LOG = LogFactory.getLog(HLog.class); private static final String HLOG_DATFILE = "hlog.dat."; - static final Text METACOLUMN = new Text("METACOLUMN:"); - static final Text METAROW = new Text("METAROW"); + static final byte [] METACOLUMN = Bytes.toBytes("METACOLUMN:"); + static final byte [] METAROW = Bytes.toBytes("METAROW"); final FileSystem fs; final Path dir; final Configuration conf; @@ -120,7 +118,8 @@ public class HLog implements HConstants { /* * Map of region to last sequence/edit id. */ - final Map lastSeqWritten = new ConcurrentHashMap(); + private final Map lastSeqWritten = Collections. + synchronizedSortedMap(new TreeMap(Bytes.BYTES_COMPARATOR)); private volatile boolean closed = false; @@ -274,8 +273,8 @@ public class HLog implements HConstants { // Now remove old log files (if any) if (LOG.isDebugEnabled()) { // Find region associated with oldest key -- helps debugging. - Text oldestRegion = null; - for (Map.Entry e: this.lastSeqWritten.entrySet()) { + byte [] oldestRegion = null; + for (Map.Entry e: this.lastSeqWritten.entrySet()) { if (e.getValue().longValue() == oldestOutstandingSeqNum.longValue()) { oldestRegion = e.getKey(); break; @@ -370,9 +369,9 @@ public class HLog implements HConstants { * @param timestamp * @throws IOException */ - void append(Text regionName, Text tableName, - TreeMap edits) throws IOException { - + void append(byte [] regionName, byte [] tableName, + TreeMap edits) + throws IOException { if (closed) { throw new IOException("Cannot append; log is closed"); } @@ -479,7 +478,7 @@ public class HLog implements HConstants { * @param logSeqId * @throws IOException */ - void completeCacheFlush(final Text regionName, final Text tableName, + void completeCacheFlush(final byte [] regionName, final byte [] tableName, final long logSeqId) throws IOException { try { @@ -535,8 +534,8 @@ public class HLog implements HConstants { } LOG.info("splitting " + logfiles.length + " log(s) in " + srcDir.toString()); - Map logWriters = - new HashMap(); + Map logWriters = + new TreeMap(Bytes.BYTES_COMPARATOR); try { for (int i = 0; i < logfiles.length; i++) { if (LOG.isDebugEnabled()) { @@ -556,17 +555,15 @@ public class HLog implements HConstants { try { int count = 0; for (; in.next(key, val); count++) { - Text tableName = key.getTablename(); - Text regionName = key.getRegionName(); + byte [] tableName = key.getTablename(); + byte [] regionName = key.getRegionName(); SequenceFile.Writer w = logWriters.get(regionName); if (w == null) { Path logfile = new Path( - HRegion.getRegionDir( - HTableDescriptor.getTableDir(rootDir, tableName), - HRegionInfo.encodeRegionName(regionName) - ), - HREGION_OLDLOGFILE_NAME - ); + HRegion.getRegionDir( + HTableDescriptor.getTableDir(rootDir, tableName), + HRegionInfo.encodeRegionName(regionName)), + HREGION_OLDLOGFILE_NAME); Path oldlogfile = null; SequenceFile.Reader old = null; if (fs.exists(logfile)) { @@ -580,7 +577,7 @@ public class HLog implements HConstants { HLogEdit.class, getCompressionType(conf)); // Use copy of regionName; regionName object is reused inside in // HStoreKey.getRegionName so its content changes as we iterate. - logWriters.put(new Text(regionName), w); + logWriters.put(regionName, w); if (LOG.isDebugEnabled()) { LOG.debug("Creating new log file writer for path " + logfile + " and region " + regionName); diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HLogEdit.java b/src/java/org/apache/hadoop/hbase/regionserver/HLogEdit.java index 49087ca7fdd..c85ea33ba4d 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HLogEdit.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HLogEdit.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.*; import java.io.*; @@ -62,7 +63,7 @@ public class HLogEdit implements Writable, HConstants { return (value == null)? false: deleteBytes.compareTo(value) == 0; } - private Text column = new Text(); + private byte [] column; private byte [] val; private long timestamp; private static final int MAX_VALUE_LEN = 128; @@ -76,18 +77,18 @@ public class HLogEdit implements Writable, HConstants { /** * Construct a fully initialized HLogEdit - * @param column column name + * @param c column name * @param bval value * @param timestamp timestamp for modification */ - public HLogEdit(Text column, byte [] bval, long timestamp) { - this.column.set(column); + public HLogEdit(byte [] c, byte [] bval, long timestamp) { + this.column = c; this.val = bval; this.timestamp = timestamp; } /** @return the column */ - public Text getColumn() { + public byte [] getColumn() { return this.column; } @@ -116,7 +117,7 @@ public class HLogEdit implements Writable, HConstants { } catch (UnsupportedEncodingException e) { throw new RuntimeException("UTF8 encoding not present?", e); } - return "(" + getColumn().toString() + "/" + getTimestamp() + "/" + + return "(" + Bytes.toString(getColumn()) + "/" + getTimestamp() + "/" + value + ")"; } @@ -124,7 +125,7 @@ public class HLogEdit implements Writable, HConstants { /** {@inheritDoc} */ public void write(DataOutput out) throws IOException { - this.column.write(out); + Bytes.writeByteArray(out, this.column); out.writeInt(this.val.length); out.write(this.val); out.writeLong(timestamp); @@ -132,7 +133,7 @@ public class HLogEdit implements Writable, HConstants { /** {@inheritDoc} */ public void readFields(DataInput in) throws IOException { - this.column.readFields(in); + this.column = Bytes.readByteArray(in); this.val = new byte[in.readInt()]; in.readFully(this.val); this.timestamp = in.readLong(); diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HLogKey.java b/src/java/org/apache/hadoop/hbase/regionserver/HLogKey.java index 496a35a9349..f77aeb4f798 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HLogKey.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HLogKey.java @@ -19,6 +19,7 @@ */ package org.apache.hadoop.hbase.regionserver; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.*; import java.io.*; @@ -31,14 +32,14 @@ import java.io.*; * also sorted. */ public class HLogKey implements WritableComparable { - Text regionName = new Text(); - Text tablename = new Text(); - Text row = new Text(); - long logSeqNum = 0L; + private byte [] regionName; + private byte [] tablename; + private byte [] row; + private long logSeqNum; /** Create an empty key useful when deserializing */ public HLogKey() { - super(); + this(null, null, null, 0L); } /** @@ -51,11 +52,11 @@ public class HLogKey implements WritableComparable { * @param row - row key * @param logSeqNum - log sequence number */ - public HLogKey(Text regionName, Text tablename, Text row, long logSeqNum) { - // TODO: Is this copy of the instances necessary? They are expensive. - this.regionName.set(regionName); - this.tablename.set(tablename); - this.row.set(row); + public HLogKey(final byte [] regionName, final byte [] tablename, + final byte [] row, long logSeqNum) { + this.regionName = regionName; + this.tablename = tablename; + this.row = row; this.logSeqNum = logSeqNum; } @@ -63,15 +64,15 @@ public class HLogKey implements WritableComparable { // A bunch of accessors ////////////////////////////////////////////////////////////////////////////// - Text getRegionName() { + byte [] getRegionName() { return regionName; } - Text getTablename() { + byte [] getTablename() { return tablename; } - Text getRow() { + byte [] getRow() { return row; } @@ -84,7 +85,8 @@ public class HLogKey implements WritableComparable { */ @Override public String toString() { - return tablename + "/" + regionName + "/" + row + "/" + logSeqNum; + return Bytes.toString(tablename) + "/" + Bytes.toString(regionName) + "/" + + Bytes.toString(row) + "/" + logSeqNum; } /** @@ -115,10 +117,10 @@ public class HLogKey implements WritableComparable { */ public int compareTo(Object o) { HLogKey other = (HLogKey) o; - int result = this.regionName.compareTo(other.regionName); + int result = Bytes.compareTo(this.regionName, other.regionName); if(result == 0) { - result = this.row.compareTo(other.row); + result = Bytes.compareTo(this.row, other.row); if(result == 0) { @@ -141,9 +143,9 @@ public class HLogKey implements WritableComparable { * {@inheritDoc} */ public void write(DataOutput out) throws IOException { - this.regionName.write(out); - this.tablename.write(out); - this.row.write(out); + Bytes.writeByteArray(out, this.regionName); + Bytes.writeByteArray(out, this.tablename); + Bytes.writeByteArray(out, this.row); out.writeLong(logSeqNum); } @@ -151,9 +153,9 @@ public class HLogKey implements WritableComparable { * {@inheritDoc} */ public void readFields(DataInput in) throws IOException { - this.regionName.readFields(in); - this.tablename.readFields(in); - this.row.readFields(in); + this.regionName = Bytes.readByteArray(in); + this.tablename = Bytes.readByteArray(in); + this.row = Bytes.readByteArray(in); this.logSeqNum = in.readLong(); } } \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 7cfca84e6cd..cfba440c20a 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -21,14 +21,14 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.Random; +import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; -import java.util.HashMap; import java.util.TreeSet; import java.util.Map.Entry; import java.util.concurrent.ConcurrentHashMap; @@ -43,12 +43,23 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.DroppedSnapshotException; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HStoreKey; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.NotServingRegionException; +import org.apache.hadoop.hbase.WrongRegionException; import org.apache.hadoop.hbase.filter.RowFilterInterface; import org.apache.hadoop.hbase.io.BatchOperation; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.io.HbaseMapWritable; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.io.Text; @@ -56,17 +67,6 @@ import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.StringUtils; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HStoreKey; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.DroppedSnapshotException; -import org.apache.hadoop.hbase.NotServingRegionException; -import org.apache.hadoop.hbase.WrongRegionException; -import org.apache.hadoop.hbase.ipc.HRegionInterface; - /** * HRegion stores data for a certain region of a table. It stores all columns * for each row. A given table consists of one or more HRegions. @@ -132,12 +132,12 @@ public class HRegion implements HConstants { } // A's start key is null but B's isn't. Assume A comes before B } else if ((srcB.getStartKey() == null) // A is not null but B is - || (srcA.getStartKey().compareTo(srcB.getStartKey()) > 0)) { // A > B + || (Bytes.compareTo(srcA.getStartKey(), srcB.getStartKey()) > 0)) { // A > B a = srcB; b = srcA; } - if (! a.getEndKey().equals(b.getStartKey())) { + if (!Bytes.equals(a.getEndKey(), b.getStartKey())) { throw new IOException("Cannot merge non-adjacent regions"); } return merge(a, b); @@ -152,8 +152,8 @@ public class HRegion implements HConstants { * @throws IOException */ public static HRegion merge(HRegion a, HRegion b) throws IOException { - if (!a.getRegionInfo().getTableDesc().getName().equals( - b.getRegionInfo().getTableDesc().getName())) { + if (!a.getRegionInfo().getTableDesc().getNameAsString().equals( + b.getRegionInfo().getTableDesc().getNameAsString())) { throw new IOException("Regions do not belong to the same table"); } FileSystem fs = a.getFilesystem(); @@ -167,12 +167,12 @@ public class HRegion implements HConstants { a.compactStores(true); if (LOG.isDebugEnabled()) { - LOG.debug("Files for region: " + a.getRegionName()); + LOG.debug("Files for region: " + a); listPaths(fs, a.getRegionDir()); } b.compactStores(true); if (LOG.isDebugEnabled()) { - LOG.debug("Files for region: " + b.getRegionName()); + LOG.debug("Files for region: " + b); listPaths(fs, b.getRegionDir()); } @@ -180,38 +180,38 @@ public class HRegion implements HConstants { HTableDescriptor tabledesc = a.getTableDesc(); HLog log = a.getLog(); Path basedir = a.getBaseDir(); - Text startKey = a.getStartKey().equals(EMPTY_TEXT) || - b.getStartKey().equals(EMPTY_TEXT) ? EMPTY_TEXT : - a.getStartKey().compareTo(b.getStartKey()) <= 0 ? + final byte [] startKey = Bytes.equals(a.getStartKey(), EMPTY_BYTE_ARRAY) || + Bytes.equals(b.getStartKey(), EMPTY_BYTE_ARRAY) ? EMPTY_BYTE_ARRAY : + Bytes.compareTo(a.getStartKey(), b.getStartKey()) <= 0 ? a.getStartKey() : b.getStartKey(); - Text endKey = a.getEndKey().equals(EMPTY_TEXT) || - b.getEndKey().equals(EMPTY_TEXT) ? EMPTY_TEXT : - a.getEndKey().compareTo(b.getEndKey()) <= 0 ? + final byte [] endKey = Bytes.equals(a.getEndKey(), EMPTY_BYTE_ARRAY) || + Bytes.equals(b.getEndKey(), EMPTY_BYTE_ARRAY) ? EMPTY_BYTE_ARRAY : + Bytes.compareTo(a.getEndKey(), b.getEndKey()) <= 0 ? b.getEndKey() : a.getEndKey(); HRegionInfo newRegionInfo = new HRegionInfo(tabledesc, startKey, endKey); LOG.info("Creating new region " + newRegionInfo.toString()); - String encodedRegionName = newRegionInfo.getEncodedName(); - Path newRegionDir = HRegion.getRegionDir(a.getBaseDir(), encodedRegionName); + int encodedName = newRegionInfo.getEncodedName(); + Path newRegionDir = HRegion.getRegionDir(a.getBaseDir(), encodedName); if(fs.exists(newRegionDir)) { throw new IOException("Cannot merge; target file collision at " + newRegionDir); } fs.mkdirs(newRegionDir); - LOG.info("starting merge of regions: " + a.getRegionName() + " and " + - b.getRegionName() + " into new region " + newRegionInfo.toString() + + LOG.info("starting merge of regions: " + a + " and " + b + + " into new region " + newRegionInfo.toString() + " with start key <" + startKey + "> and end key <" + endKey + ">"); // Move HStoreFiles under new region directory - Map> byFamily = - new TreeMap>(); + Map> byFamily = + new TreeMap>(Bytes.BYTES_COMPARATOR); byFamily = filesByFamily(byFamily, a.close()); byFamily = filesByFamily(byFamily, b.close()); - for (Map.Entry> es : byFamily.entrySet()) { - Text colFamily = es.getKey(); - makeColumnFamilyDirs(fs, basedir, encodedRegionName, colFamily, tabledesc); + for (Map.Entry> es : byFamily.entrySet()) { + byte [] colFamily = es.getKey(); + makeColumnFamilyDirs(fs, basedir, encodedName, colFamily, tabledesc); // Because we compacted the source regions we should have no more than two // HStoreFiles per family and there will be no reference store @@ -252,7 +252,7 @@ public class HRegion implements HConstants { deleteRegion(fs, a.getRegionDir()); deleteRegion(fs, b.getRegionDir()); - LOG.info("merge completed. New region is " + dstRegion.getRegionName()); + LOG.info("merge completed. New region is " + dstRegion); return dstRegion; } @@ -263,11 +263,11 @@ public class HRegion implements HConstants { * @param storeFiles Store files to process. * @return Returns byFamily */ - private static Map> filesByFamily( - Map> byFamily, List storeFiles) { - for(HStoreFile src: storeFiles) { + private static Map> filesByFamily( + Map> byFamily, List storeFiles) { + for (HStoreFile src: storeFiles) { List v = byFamily.get(src.getColFamily()); - if(v == null) { + if (v == null) { v = new ArrayList(); byFamily.put(src.getColFamily(), v); } @@ -312,14 +312,14 @@ public class HRegion implements HConstants { // Members ////////////////////////////////////////////////////////////////////////////// - volatile Map rowsToLocks = new ConcurrentHashMap(); - volatile Map locksToRows = new ConcurrentHashMap(); - volatile Map stores = new ConcurrentHashMap(); - volatile Map> targetColumns = - new ConcurrentHashMap>(); - - final AtomicLong memcacheSize = new AtomicLong(0); + private final Map locksToRows = + new ConcurrentHashMap(); + private final Map> targetColumns = + new ConcurrentHashMap>(); private volatile boolean flushRequested; + // Default access because read by tests. + final Map stores = new ConcurrentHashMap(); + final AtomicLong memcacheSize = new AtomicLong(0); final Path basedir; final HLog log; @@ -426,29 +426,29 @@ public class HRegion implements HConstants { this.flushListener = flushListener; this.flushRequested = false; this.threadWakeFrequency = conf.getLong(THREAD_WAKE_FREQUENCY, 10 * 1000); - this.regiondir = new Path(basedir, this.regionInfo.getEncodedName()); + String encodedNameStr = Integer.toString(this.regionInfo.getEncodedName()); + this.regiondir = new Path(basedir, encodedNameStr); Path oldLogFile = new Path(regiondir, HREGION_OLDLOGFILE_NAME); if (LOG.isDebugEnabled()) { - LOG.debug("Opening region " + this.regionInfo.getRegionName() + "/" + + LOG.debug("Opening region " + this + "/" + this.regionInfo.getEncodedName()); } this.regionCompactionDir = - new Path(getCompactionDir(basedir), this.regionInfo.getEncodedName()); + new Path(getCompactionDir(basedir), encodedNameStr); // Move prefab HStore files into place (if any). This picks up split files // and any merges from splits and merges dirs. - if(initialFiles != null && fs.exists(initialFiles)) { + if (initialFiles != null && fs.exists(initialFiles)) { fs.rename(initialFiles, this.regiondir); } // Load in all the HStores. long maxSeqId = -1; - for(HColumnDescriptor c : - this.regionInfo.getTableDesc().families().values()) { + for (HColumnDescriptor c : this.regionInfo.getTableDesc().getFamilies()) { HStore store = new HStore(this.basedir, this.regionInfo, c, this.fs, oldLogFile, this.conf, reporter); - stores.put(c.getFamilyName(), store); + stores.put(Bytes.mapKey(c.getName()), store); long storeSeqId = store.getMaxSequenceId(); if (storeSeqId > maxSeqId) { maxSeqId = storeSeqId; @@ -464,8 +464,9 @@ public class HRegion implements HConstants { // Add one to the current maximum sequence id so new edits are beyond. this.minSequenceId = maxSeqId + 1; if (LOG.isDebugEnabled()) { - LOG.debug("Next sequence id for region " + regionInfo.getRegionName() + - " is " + this.minSequenceId); + LOG.debug("Next sequence id for region " + + Bytes.toString(regionInfo.getRegionName()) + " is " + + this.minSequenceId); } // Get rid of any splits or merges that were lost in-progress @@ -488,8 +489,8 @@ public class HRegion implements HConstants { // HRegion is ready to go! this.writestate.compacting = false; this.lastFlushTime = System.currentTimeMillis(); - LOG.info("region " + this.regionInfo.getRegionName() + "/" + - this.regionInfo.getEncodedName() + " available"); + LOG.info("region " + this + "/" + this.regionInfo.getEncodedName() + + " available"); } /** @@ -544,9 +545,8 @@ public class HRegion implements HConstants { */ List close(boolean abort, final RegionUnavailableListener listener) throws IOException { - Text regionName = this.regionInfo.getRegionName(); if (isClosed()) { - LOG.warn("region " + regionName + " already closed"); + LOG.warn("region " + this + " already closed"); return null; } synchronized (splitLock) { @@ -554,16 +554,13 @@ public class HRegion implements HConstants { // Disable compacting and flushing by background threads for this // region. writestate.writesEnabled = false; - LOG.debug("compactions and cache flushes disabled for region " + - regionName); + LOG.debug("Compactions and cache flushes disabled for region " + this); while (writestate.compacting || writestate.flushing) { LOG.debug("waiting for" + (writestate.compacting ? " compaction" : "") + (writestate.flushing ? (writestate.compacting ? "," : "") + " cache flush" : - "" - ) + " to complete for region " + regionName - ); + "") + " to complete for region " + this); try { writestate.wait(); } catch (InterruptedException iex) { @@ -572,7 +569,7 @@ public class HRegion implements HConstants { } } splitsAndClosesLock.writeLock().lock(); - LOG.debug("Updates and scanners for region " + regionName + " disabled"); + LOG.debug("Updates and scanners disabled for region " + this); try { // Wait for active scanners to finish. The write lock we hold will // prevent new scanners from being created. @@ -587,14 +584,14 @@ public class HRegion implements HConstants { } } } - LOG.debug("no more active scanners for region " + regionName); + LOG.debug("No more active scanners for region " + this); // Write lock means no more row locks can be given out. Wait on // outstanding row locks to come in before we close so we do not drop // outstanding updates. waitOnRowLocks(); - LOG.debug("no more row locks outstanding on region " + regionName); - + LOG.debug("No more row locks outstanding on region " + this); + if (listener != null) { // If there is a listener, let them know that we have now // acquired all the necessary locks and are starting to @@ -619,7 +616,7 @@ public class HRegion implements HConstants { listener.closed(getRegionName()); } - LOG.info("closed " + this.regionInfo.getRegionName()); + LOG.info("closed " + this); return result; } finally { splitsAndClosesLock.writeLock().unlock(); @@ -632,12 +629,12 @@ public class HRegion implements HConstants { ////////////////////////////////////////////////////////////////////////////// /** @return start key for region */ - public Text getStartKey() { + public byte [] getStartKey() { return this.regionInfo.getStartKey(); } /** @return end key for region */ - public Text getEndKey() { + public byte [] getEndKey() { return this.regionInfo.getEndKey(); } @@ -647,7 +644,7 @@ public class HRegion implements HConstants { } /** @return region name */ - public Text getRegionName() { + public byte [] getRegionName() { return this.regionInfo.getRegionName(); } @@ -716,37 +713,39 @@ public class HRegion implements HConstants { * @throws IOException */ HRegion[] splitRegion(final RegionUnavailableListener listener, - final Text midKey) throws IOException { + final byte [] midKey) throws IOException { synchronized (splitLock) { if (closed.get()) { return null; } // Add start/end key checking: hbase-428. - Text startKey = new Text(this.regionInfo.getStartKey()); - Text endKey = new Text(this.regionInfo.getEndKey()); - if (startKey.equals(midKey)) { + byte [] startKey = this.regionInfo.getStartKey(); + byte [] endKey = this.regionInfo.getEndKey(); + if (Bytes.equals(startKey, midKey)) { LOG.debug("Startkey (" + startKey + ") and midkey + (" + midKey + ") are same, not splitting"); return null; } - if (midKey.equals(endKey)) { + if (Bytes.equals(midKey, endKey)) { LOG.debug("Endkey and midkey are same, not splitting"); return null; } - LOG.info("Starting split of region " + getRegionName()); + LOG.info("Starting split of region " + this); Path splits = new Path(this.regiondir, SPLITDIR); if(!this.fs.exists(splits)) { this.fs.mkdirs(splits); } HRegionInfo regionAInfo = new HRegionInfo(this.regionInfo.getTableDesc(), startKey, midKey); - Path dirA = new Path(splits, regionAInfo.getEncodedName()); + Path dirA = + new Path(splits, Integer.toString(regionAInfo.getEncodedName())); if(fs.exists(dirA)) { throw new IOException("Cannot split; target file collision at " + dirA); } HRegionInfo regionBInfo = new HRegionInfo(this.regionInfo.getTableDesc(), midKey, endKey); - Path dirB = new Path(splits, regionBInfo.getEncodedName()); + Path dirB = + new Path(splits, Integer.toString(regionBInfo.getEncodedName())); if(this.fs.exists(dirB)) { throw new IOException("Cannot split; target file collision at " + dirB); } @@ -844,7 +843,7 @@ public class HRegion implements HConstants { * @return mid key if split is needed * @throws IOException */ - public Text compactStores() throws IOException { + public byte [] compactStores() throws IOException { return compactStores(false); } @@ -864,8 +863,8 @@ public class HRegion implements HConstants { * @return mid key if split is needed * @throws IOException */ - private Text compactStores(final boolean force) throws IOException { - Text midKey = null; + private byte [] compactStores(final boolean force) throws IOException { + byte [] midKey = null; if (this.closed.get()) { return midKey; } @@ -874,23 +873,23 @@ public class HRegion implements HConstants { if (!writestate.compacting && writestate.writesEnabled) { writestate.compacting = true; } else { - LOG.info("NOT compacting region " + getRegionName() + + LOG.info("NOT compacting region " + this + ": compacting=" + writestate.compacting + ", writesEnabled=" + writestate.writesEnabled); return midKey; } } - LOG.info("checking compaction on region " + getRegionName()); + LOG.info("starting compaction on region " + this); long startTime = System.currentTimeMillis(); doRegionCompactionPrep(); for (HStore store: stores.values()) { - Text key = store.compact(force); + final byte [] key = store.compact(force); if (key != null && midKey == null) { midKey = key; } } doRegionCompactionCleanup(); - LOG.info("compaction checking completed on region " + getRegionName() + " in " + + LOG.info("compaction completed on region " + this + " in " + StringUtils.formatTimeDiff(System.currentTimeMillis(), startTime)); } finally { synchronized (writestate) { @@ -930,8 +929,8 @@ public class HRegion implements HConstants { writestate.flushing = true; } else { if(LOG.isDebugEnabled()) { - LOG.debug("NOT flushing memcache for region " + - this.regionInfo.getRegionName() + ", flushing=" + + LOG.debug("NOT flushing memcache for region " + this + + ", flushing=" + writestate.flushing + ", writesEnabled=" + writestate.writesEnabled); } @@ -996,8 +995,8 @@ public class HRegion implements HConstants { this.lastFlushTime = startTime; if (LOG.isDebugEnabled()) { - LOG.debug("Started memcache flush for region " + - this.regionInfo.getRegionName() + ". Current region memcache size " + + LOG.debug("Started memcache flush for region " + this + + ". Current region memcache size " + StringUtils.humanReadableInt(this.memcacheSize.get())); } @@ -1049,7 +1048,7 @@ 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.getRegionName(), + this.log.completeCacheFlush(getRegionName(), regionInfo.getTableDesc().getName(), sequenceId); // C. Finally notify anyone waiting on memcache to clear: @@ -1059,8 +1058,8 @@ public class HRegion implements HConstants { } if (LOG.isDebugEnabled()) { - LOG.debug("Finished memcache flush for region " + - this.regionInfo.getRegionName() + " in " + + LOG.debug("Finished memcache flush for region " + this + + " in " + (System.currentTimeMillis() - startTime) + "ms, sequence id=" + sequenceId); } @@ -1078,7 +1077,7 @@ public class HRegion implements HConstants { * @return column value * @throws IOException */ - public Cell get(Text row, Text column) throws IOException { + public Cell get(byte [] row, byte [] column) throws IOException { Cell[] results = get(row, column, Long.MAX_VALUE, 1); return (results == null || results.length == 0)? null: results[0]; } @@ -1091,7 +1090,8 @@ public class HRegion implements HConstants { * @return array of values one element per version * @throws IOException */ - public Cell[] get(Text row, Text column, int numVersions) throws IOException { + public Cell[] get(byte [] row, byte [] column, int numVersions) + throws IOException { return get(row, column, Long.MAX_VALUE, numVersions); } @@ -1105,23 +1105,19 @@ public class HRegion implements HConstants { * @return array of values one element per version that matches the timestamp * @throws IOException */ - public Cell[] get(Text row, Text column, long timestamp, int numVersions) - throws IOException { - + public Cell[] get(byte [] row, byte [] column, long timestamp, + int numVersions) + throws IOException { if (this.closed.get()) { - throw new IOException("Region " + this.getRegionName().toString() + - " closed"); + throw new IOException("Region " + this + " closed"); } // Make sure this is a valid row and valid column checkRow(row); checkColumn(column); - // Don't need a row lock for a simple get - HStoreKey key = new HStoreKey(row, column, timestamp); - HStore targetStore = stores.get(HStoreKey.extractFamily(column)); - return targetStore.get(key, numVersions); + return getStore(column).get(key, numVersions); } /** @@ -1140,20 +1136,20 @@ public class HRegion implements HConstants { * @return Map values * @throws IOException */ - public Map getFull(final Text row, final Set columns, - final long ts) + public Map getFull(final byte [] row, + final Set columns, final long ts) throws IOException { HStoreKey key = new HStoreKey(row, ts); - obtainRowLock(row); + Integer lid = obtainRowLock(row); try { - TreeMap result = new TreeMap(); - for (Text colFamily: stores.keySet()) { - HStore targetStore = stores.get(colFamily); + TreeMap result = + new TreeMap(Bytes.BYTES_COMPARATOR); + for (HStore targetStore: stores.values()) { targetStore.getFull(key, columns, result); } return result; } finally { - releaseRowLock(row); + releaseRowLock(lid); } } @@ -1166,50 +1162,40 @@ public class HRegion implements HConstants { * @return map of values * @throws IOException */ - public RowResult getClosestRowBefore(final Text row) + public RowResult getClosestRowBefore(final byte [] row) throws IOException{ // look across all the HStores for this region and determine what the // closest key is across all column families, since the data may be sparse - HStoreKey key = null; checkRow(row); splitsAndClosesLock.readLock().lock(); try { // examine each column family for the preceeding or matching key - for(Text colFamily : stores.keySet()){ - HStore store = stores.get(colFamily); - + for (HStore store : stores.values()) { // get the closest key - Text closestKey = store.getRowKeyAtOrBefore(row); - + byte [] closestKey = store.getRowKeyAtOrBefore(row); // if it happens to be an exact match, we can stop looping - if (row.equals(closestKey)) { + if (Bytes.equals(row, closestKey)) { key = new HStoreKey(closestKey); break; } - // otherwise, we need to check if it's the max and move to the next if (closestKey != null - && (key == null || closestKey.compareTo(key.getRow()) > 0) ) { + && (key == null || Bytes.compareTo(closestKey, key.getRow()) > 0) ) { key = new HStoreKey(closestKey); } } - if (key == null) { return null; } // now that we've found our key, get the values - Map cells = new HashMap(); - for (Text colFamily: stores.keySet()) { - HStore targetStore = stores.get(colFamily); - targetStore.getFull(key, null, cells); + HbaseMapWritable cells = + new HbaseMapWritable(); + for (HStore s: stores.values()) { + s.getFull(key, null, cells); } - - HbaseMapWritable cellsWritten = new HbaseMapWritable(); - cellsWritten.putAll(cells); - - return new RowResult(key.getRow(), cellsWritten); + return new RowResult(key.getRow(), cells); } finally { splitsAndClosesLock.readLock().unlock(); } @@ -1225,16 +1211,26 @@ public class HRegion implements HConstants { * @return Ordered list of versions keys going from newest back. * @throws IOException */ - private List getKeys(final HStoreKey origin, final int versions) - throws IOException { - - List keys = null; - Text colFamily = HStoreKey.extractFamily(origin.getColumn()); - HStore targetStore = stores.get(colFamily); - if (targetStore != null) { - // Pass versions without modification since in the store getKeys, it - // includes the size of the passed keys array when counting. - keys = targetStore.getKeys(origin, versions); + private Set getKeys(final HStoreKey origin, final int versions) + throws IOException { + Set keys = new TreeSet(); + Collection storesToCheck = null; + if (origin.getColumn() == null || origin.getColumn().length == 0) { + // All families + storesToCheck = this.stores.values(); + } else { + storesToCheck = new ArrayList(1); + storesToCheck.add(getStore(origin.getColumn())); + } + for (HStore targetStore: storesToCheck) { + if (targetStore != null) { + // Pass versions without modification since in the store getKeys, it + // includes the size of the passed keys array when counting. + List r = targetStore.getKeys(origin, versions); + if (r != null) { + keys.addAll(r); + } + } } return keys; } @@ -1255,27 +1251,25 @@ public class HRegion implements HConstants { * @return InternalScanner * @throws IOException */ - public InternalScanner getScanner(Text[] cols, Text firstRow, + public InternalScanner getScanner(byte[][] cols, byte [] firstRow, long timestamp, RowFilterInterface filter) throws IOException { splitsAndClosesLock.readLock().lock(); try { if (this.closed.get()) { - throw new IOException("Region " + this.getRegionName().toString() + - " closed"); + throw new IOException("Region " + this + " closed"); } - TreeSet families = new TreeSet(); - for(int i = 0; i < cols.length; i++) { - families.add(HStoreKey.extractFamily(cols[i])); + TreeSet families = new TreeSet(Bytes.BYTES_COMPARATOR); + for (int i = 0; i < cols.length; i++) { + families.add(HStoreKey.getFamily(cols[i])); } List storelist = new ArrayList(); - for (Text family: families) { - HStore s = stores.get(family); + for (byte [] family: families) { + HStore s = stores.get(Bytes.mapKey(family)); if (s == null) { continue; } - storelist.add(stores.get(family)); - + storelist.add(s); } return new HScanner(cols, firstRow, timestamp, storelist.toArray(new HStore [storelist.size()]), filter); @@ -1293,7 +1287,8 @@ public class HRegion implements HConstants { * @throws IOException */ public void batchUpdate(BatchUpdate b) - throws IOException { + throws IOException { + // Do a rough check that we have resources to accept a write. The check is // 'rough' in that between the resource check and the call to obtain a // read lock, resources may run out. For now, the thought is that this @@ -1305,14 +1300,12 @@ public class HRegion implements HConstants { // #commit or #abort or if the HRegionServer lease on the lock expires. // See HRegionServer#RegionListener for how the expire on HRegionServer // invokes a HRegion#abort. - Text row = b.getRow(); - long lockid = obtainRowLock(row); - + byte [] row = b.getRow(); + Integer lid = obtainRowLock(row); long commitTime = (b.getTimestamp() == LATEST_TIMESTAMP) ? - System.currentTimeMillis() : b.getTimestamp(); - + System.currentTimeMillis() : b.getTimestamp(); try { - List deletes = null; + List deletes = null; for (BatchOperation op: b) { HStoreKey key = new HStoreKey(row, op.getColumn(), commitTime); byte[] val = null; @@ -1325,7 +1318,7 @@ public class HRegion implements HConstants { if (b.getTimestamp() == LATEST_TIMESTAMP) { // Save off these deletes if (deletes == null) { - deletes = new ArrayList(); + deletes = new ArrayList(); } deletes.add(op.getColumn()); } else { @@ -1333,28 +1326,27 @@ public class HRegion implements HConstants { } } if (val != null) { - localput(lockid, key, val); + localput(lid, key, val); } } TreeMap edits = - this.targetColumns.remove(Long.valueOf(lockid)); + this.targetColumns.remove(lid); + if (edits != null && edits.size() > 0) { update(edits); } if (deletes != null && deletes.size() > 0) { // We have some LATEST_TIMESTAMP deletes to run. - for (Text column: deletes) { + for (byte [] column: deletes) { deleteMultiple(row, column, LATEST_TIMESTAMP, 1); } } - } catch (IOException e) { - this.targetColumns.remove(Long.valueOf(lockid)); + this.targetColumns.remove(Long.valueOf(lid)); throw e; - } finally { - releaseRowLock(row); + releaseRowLock(lid); } } @@ -1389,7 +1381,7 @@ public class HRegion implements HConstants { } } if (blocked) { - LOG.info("Unblocking updates for region " + getRegionName() + " '" + + LOG.info("Unblocking updates for region " + this + " '" + Thread.currentThread().getName() + "'"); } } @@ -1401,15 +1393,14 @@ public class HRegion implements HConstants { * @param ts Delete all entries that have this timestamp or older * @throws IOException */ - public void deleteAll(final Text row, final Text column, final long ts) - throws IOException { - + public void deleteAll(final byte [] row, final byte [] column, final long ts) + throws IOException { checkColumn(column); - obtainRowLock(row); + Integer lid = obtainRowLock(row); try { deleteMultiple(row, column, ts, ALL_VERSIONS); } finally { - releaseRowLock(row); + releaseRowLock(lid); } } @@ -1419,15 +1410,13 @@ public class HRegion implements HConstants { * @param ts Delete all entries that have this timestamp or older * @throws IOException */ - public void deleteAll(final Text row, final long ts) - throws IOException { - - obtainRowLock(row); - + public void deleteAll(final byte [] row, final long ts) + throws IOException { + Integer lid = obtainRowLock(row); try { - for(Map.Entry store : stores.entrySet()){ - List keys = store.getValue().getKeys(new HStoreKey(row, ts), ALL_VERSIONS); - + for (HStore store : stores.values()){ + List keys = store.getKeys(new HStoreKey(row, ts), + ALL_VERSIONS); TreeMap edits = new TreeMap(); for (HStoreKey key: keys) { edits.put(key, HLogEdit.deleteBytes.get()); @@ -1435,7 +1424,7 @@ public class HRegion implements HConstants { update(edits); } } finally { - releaseRowLock(row); + releaseRowLock(lid); } } @@ -1448,16 +1437,14 @@ public class HRegion implements HConstants { * @param timestamp Timestamp to match * @throws IOException */ - public void deleteFamily(Text row, Text family, long timestamp) + public void deleteFamily(byte [] row, byte [] family, long timestamp) throws IOException{ - obtainRowLock(row); - + Integer lid = obtainRowLock(row); try { // find the HStore for the column family - HStore store = stores.get(HStoreKey.extractFamily(family)); + HStore store = getStore(family); // find all the keys that match our criteria List keys = store.getKeys(new HStoreKey(row, timestamp), ALL_VERSIONS); - // delete all the cells TreeMap edits = new TreeMap(); for (HStoreKey key: keys) { @@ -1465,13 +1452,13 @@ public class HRegion implements HConstants { } update(edits); } finally { - releaseRowLock(row); + releaseRowLock(lid); } } /** * Delete one or many cells. - * Used to support {@link #deleteAll(Text, Text, long)} and deletion of + * Used to support {@link #deleteAll(byte [], byte [], long)} and deletion of * latest cell. * * @param row @@ -1481,11 +1468,11 @@ public class HRegion implements HConstants { * {@link HConstants#ALL_VERSIONS} to delete all. * @throws IOException */ - private void deleteMultiple(final Text row, final Text column, final long ts, - final int versions) throws IOException { - + private void deleteMultiple(final byte [] row, final byte [] column, + final long ts, final int versions) + throws IOException { HStoreKey origin = new HStoreKey(row, column, ts); - List keys = getKeys(origin, versions); + Set keys = getKeys(origin, versions); if (keys.size() > 0) { TreeMap edits = new TreeMap(); for (HStoreKey key: keys) { @@ -1507,15 +1494,14 @@ public class HRegion implements HConstants { * @param val Value to enter into cell * @throws IOException */ - private void localput(final long lockid, final HStoreKey key, - final byte [] val) throws IOException { - + private void localput(final Integer lockid, final HStoreKey key, + final byte [] val) + throws IOException { checkColumn(key.getColumn()); - Long lid = Long.valueOf(lockid); - TreeMap targets = this.targetColumns.get(lid); + TreeMap targets = this.targetColumns.get(lockid); if (targets == null) { targets = new TreeMap(); - this.targetColumns.put(lid, targets); + this.targetColumns.put(lockid, targets); } targets.put(key, val); } @@ -1543,7 +1529,7 @@ public class HRegion implements HConstants { HStoreKey key = e.getKey(); byte[] val = e.getValue(); size = this.memcacheSize.addAndGet(getEntrySize(key, val)); - stores.get(HStoreKey.extractFamily(key.getColumn())).add(key, val); + getStore(key.getColumn()).add(key, val); } flush = this.flushListener != null && !this.flushRequested && size > this.memcacheFlushSize; @@ -1557,6 +1543,15 @@ public class HRegion implements HConstants { } } + /* + * @param column + * @return Store that goes with the family on passed column. + * TODO: Make this lookup faster. + */ + private HStore getStore(final byte [] column) { + return this.stores.get(HStoreKey.getFamilyMapKey(column)); + } + /* * Calculate size of passed key/value pair. * Used here when we update region to figure what to add to this.memcacheSize @@ -1575,12 +1570,13 @@ public class HRegion implements HConstants { ////////////////////////////////////////////////////////////////////////////// /** Make sure this is a valid row for the HRegion */ - private void checkRow(Text row) throws IOException { + private void checkRow(final byte [] row) throws IOException { if(!rowIsInRange(regionInfo, row)) { throw new WrongRegionException("Requested row out of range for " + - "HRegion " + regionInfo.getRegionName() + ", startKey='" + - regionInfo.getStartKey() + "', getEndKey()='" + - regionInfo.getEndKey() + "', row='" + row + "'"); + "HRegion " + this + ", startKey='" + + Bytes.toString(regionInfo.getStartKey()) + "', getEndKey()='" + + Bytes.toString(regionInfo.getEndKey()) + "', row='" + + Bytes.toString(row) + "'"); } } @@ -1589,12 +1585,14 @@ public class HRegion implements HConstants { * @param columnName * @throws IOException */ - private void checkColumn(Text columnName) throws IOException { - Text family = HStoreKey.extractFamily(columnName, true); - if (!regionInfo.getTableDesc().hasFamily(family)) { - throw new IOException("Requested column family " + family - + " does not exist in HRegion " + regionInfo.getRegionName() - + " for table " + regionInfo.getTableDesc().getName()); + private void checkColumn(final byte [] columnName) throws IOException { + if (columnName == null) { + return; + } + if (!regionInfo.getTableDesc().hasFamily(columnName)) { + throw new IOException("Column family on " + + Bytes.toString(columnName) + " does not exist in region " + this + + " in table " + regionInfo.getTableDesc()); } } @@ -1621,59 +1619,57 @@ public class HRegion implements HConstants { * @throws IOException * @return The id of the held lock. */ - long obtainRowLock(Text row) throws IOException { + Integer obtainRowLock(final byte [] row) throws IOException { checkRow(row); splitsAndClosesLock.readLock().lock(); try { if (this.closed.get()) { - throw new NotServingRegionException("Region " + - this.getRegionName().toString() + " closed"); + throw new NotServingRegionException("Region " + this + " closed"); } - synchronized (rowsToLocks) { - while (rowsToLocks.get(row) != null) { + Integer key = Bytes.mapKey(row); + synchronized (locksToRows) { + while (locksToRows.containsKey(key)) { try { - rowsToLocks.wait(); + locksToRows.wait(); } catch (InterruptedException ie) { // Empty } } - Long lid = Long.valueOf(Math.abs(rand.nextLong())); - rowsToLocks.put(row, lid); - locksToRows.put(lid, row); - rowsToLocks.notifyAll(); - return lid.longValue(); + locksToRows.put(key, row); + locksToRows.notifyAll(); + return key; } } finally { splitsAndClosesLock.readLock().unlock(); } } - Text getRowFromLock(long lockid) { - // Pattern is that all access to rowsToLocks and/or to - // locksToRows is via a lock on rowsToLocks. - synchronized (rowsToLocks) { - return locksToRows.get(Long.valueOf(lockid)); - } + /** + * Used by unit tests. + * @param lockid + * @return Row that goes with lockid + */ + byte [] getRowFromLock(final Integer lockid) { + return locksToRows.get(lockid); } /** * Release the row lock! * @param row Name of row whose lock we are to release */ - void releaseRowLock(Text row) { - synchronized (rowsToLocks) { - long lockid = rowsToLocks.remove(row).longValue(); - locksToRows.remove(Long.valueOf(lockid)); - rowsToLocks.notifyAll(); + void releaseRowLock(final Integer lockid) { + synchronized (locksToRows) { + locksToRows.remove(lockid); + locksToRows.notifyAll(); } } private void waitOnRowLocks() { - synchronized (rowsToLocks) { - while (this.rowsToLocks.size() > 0) { - LOG.debug("waiting for " + this.rowsToLocks.size() + " row locks"); + synchronized (locksToRows) { + while (this.locksToRows.size() > 0) { + LOG.debug("waiting for " + this.locksToRows.size() + " row locks"); try { - this.rowsToLocks.wait(); + this.locksToRows.wait(); } catch (InterruptedException e) { // Catch. Let while test determine loop-end. } @@ -1696,7 +1692,7 @@ public class HRegion implements HConstants { /** {@inheritDoc} */ @Override public String toString() { - return regionInfo.getRegionName().toString(); + return this.regionInfo.getRegionNameAsString(); } /** @return Path of region base directory */ @@ -1709,13 +1705,13 @@ public class HRegion implements HConstants { */ private class HScanner implements InternalScanner { private InternalScanner[] scanners; - private TreeMap[] resultSets; + private TreeMap[] resultSets; private HStoreKey[] keys; private RowFilterInterface filter; /** Create an HScanner with a handle on many HStores. */ @SuppressWarnings("unchecked") - HScanner(Text[] cols, Text firstRow, long timestamp, HStore[] stores, + HScanner(byte [][] cols, byte [] firstRow, long timestamp, HStore[] stores, RowFilterInterface filter) throws IOException { this.filter = filter; @@ -1747,7 +1743,7 @@ public class HRegion implements HConstants { this.keys = new HStoreKey[scanners.length]; for (int i = 0; i < scanners.length; i++) { keys[i] = new HStoreKey(); - resultSets[i] = new TreeMap(); + resultSets[i] = new TreeMap(Bytes.BYTES_COMPARATOR); if(scanners[i] != null && !scanners[i].next(keys[i], resultSets[i])) { closeScanner(i); } @@ -1760,23 +1756,22 @@ public class HRegion implements HConstants { /** {@inheritDoc} */ @SuppressWarnings("null") - public boolean next(HStoreKey key, SortedMap results) + public boolean next(HStoreKey key, SortedMap results) throws IOException { boolean moreToFollow = false; boolean filtered = false; do { // Find the lowest-possible key. - - Text chosenRow = null; + byte [] chosenRow = null; long chosenTimestamp = -1; for (int i = 0; i < this.keys.length; i++) { if (scanners[i] != null && (chosenRow == null || - (keys[i].getRow().compareTo(chosenRow) < 0) || - ((keys[i].getRow().compareTo(chosenRow) == 0) && + (Bytes.compareTo(keys[i].getRow(), chosenRow) < 0) || + ((Bytes.compareTo(keys[i].getRow(), chosenRow) == 0) && (keys[i].getTimestamp() > chosenTimestamp)))) { - chosenRow = new Text(keys[i].getRow()); + chosenRow = keys[i].getRow(); chosenTimestamp = keys[i].getTimestamp(); } } @@ -1787,16 +1782,16 @@ public class HRegion implements HConstants { // Here we are setting the passed in key with current row+timestamp key.setRow(chosenRow); key.setVersion(chosenTimestamp); - key.setColumn(HConstants.EMPTY_TEXT); + key.setColumn(HConstants.EMPTY_BYTE_ARRAY); for (int i = 0; i < scanners.length; i++) { if (scanners[i] != null && - keys[i].getRow().compareTo(chosenRow) == 0) { + Bytes.compareTo(keys[i].getRow(), chosenRow) == 0) { // NOTE: We used to do results.putAll(resultSets[i]); // but this had the effect of overwriting newer // values with older ones. So now we only insert // a result if the map does not contain the key. - for (Map.Entry e : resultSets[i].entrySet()) { + for (Map.Entry e : resultSets[i].entrySet()) { if (!results.containsKey(e.getKey())) { results.put(e.getKey(), e.getValue()); } @@ -1813,7 +1808,7 @@ public class HRegion implements HConstants { // If the current scanner is non-null AND has a lower-or-equal // row label, then its timestamp is bad. We need to advance it. while ((scanners[i] != null) && - (keys[i].getRow().compareTo(chosenRow) <= 0)) { + (Bytes.compareTo(keys[i].getRow(), chosenRow) <= 0)) { resultSets[i].clear(); if (!scanners[i].next(keys[i], resultSets[i])) { closeScanner(i); @@ -1979,17 +1974,15 @@ public class HRegion implements HConstants { throws IOException { meta.checkResources(); // The row key is the region name - Text row = r.getRegionName(); - meta.obtainRowLock(row); + byte [] row = r.getRegionName(); + Integer lid = meta.obtainRowLock(row); try { - HStoreKey key = - new HStoreKey(row, COL_REGIONINFO, System.currentTimeMillis()); + HStoreKey key = new HStoreKey(row, COL_REGIONINFO, System.currentTimeMillis()); TreeMap edits = new TreeMap(); edits.put(key, Writables.getBytes(r.getRegionInfo())); meta.update(edits); - } finally { - meta.releaseRowLock(row); + meta.releaseRowLock(lid); } } @@ -2004,7 +1997,8 @@ public class HRegion implements HConstants { * @throws IOException */ public static void removeRegionFromMETA(final HRegionInterface srvr, - final Text metaRegionName, final Text regionName) throws IOException { + final byte [] metaRegionName, final byte [] regionName) + throws IOException { srvr.deleteAll(metaRegionName, regionName, HConstants.LATEST_TIMESTAMP); } @@ -2017,7 +2011,8 @@ public class HRegion implements HConstants { * @throws IOException */ public static void offlineRegionInMETA(final HRegionInterface srvr, - final Text metaRegionName, final HRegionInfo info) throws IOException { + final byte [] metaRegionName, final HRegionInfo info) + throws IOException { BatchUpdate b = new BatchUpdate(info.getRegionName()); info.setOffline(true); b.put(COL_REGIONINFO, Writables.getBytes(info)); @@ -2055,10 +2050,9 @@ public class HRegion implements HConstants { * @param tabledir qualified path for table * @param name ENCODED region name * @return Path of HRegion directory - * @see HRegionInfo#encodeRegionName(Text) */ - public static Path getRegionDir(final Path tabledir, final String name) { - return new Path(tabledir, name); + public static Path getRegionDir(final Path tabledir, final int name) { + return new Path(tabledir, Integer.toString(name)); } /** @@ -2070,9 +2064,8 @@ public class HRegion implements HConstants { */ public static Path getRegionDir(final Path rootdir, final HRegionInfo info) { return new Path( - HTableDescriptor.getTableDir(rootdir, info.getTableDesc().getName()), - info.getEncodedName() - ); + HTableDescriptor.getTableDir(rootdir, info.getTableDesc().getName()), + Integer.toString(info.getEncodedName())); } /** @@ -2083,13 +2076,13 @@ public class HRegion implements HConstants { * @param row row to be checked * @return true if the row is within the range specified by the HRegionInfo */ - public static boolean rowIsInRange(HRegionInfo info, Text row) { - return ((info.getStartKey().getLength() == 0) || - (info.getStartKey().compareTo(row) <= 0)) && - ((info.getEndKey().getLength() == 0) || - (info.getEndKey().compareTo(row) > 0)); + public static boolean rowIsInRange(HRegionInfo info, final byte [] row) { + return ((info.getStartKey().length == 0) || + (Bytes.compareTo(info.getStartKey(), row) <= 0)) && + ((info.getEndKey().length == 0) || + (Bytes.compareTo(info.getEndKey(), row) > 0)); } - + /** * Make the directories for a specific column family * @@ -2101,12 +2094,11 @@ public class HRegion implements HConstants { * @throws IOException */ public static void makeColumnFamilyDirs(FileSystem fs, Path basedir, - String encodedRegionName, Text colFamily, HTableDescriptor tabledesc) + int encodedRegionName, byte [] colFamily, HTableDescriptor tabledesc) throws IOException { fs.mkdirs(HStoreFile.getMapDir(basedir, encodedRegionName, colFamily)); fs.mkdirs(HStoreFile.getInfoDir(basedir, encodedRegionName, colFamily)); - if (tabledesc.families().get(new Text(colFamily + ":")).getBloomFilter() != - null) { + if (tabledesc.getFamily(colFamily).getBloomFilter() != null) { fs.mkdirs(HStoreFile.getFilterDir(basedir, encodedRegionName, colFamily)); } } diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index a23da53b05a..782cdf55a30 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -24,8 +24,9 @@ import java.lang.Thread.UncaughtExceptionHandler; import java.lang.reflect.Constructor; import java.net.InetSocketAddress; import java.net.UnknownHostException; -import java.util.Arrays; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -35,6 +36,7 @@ import java.util.Map; import java.util.Random; import java.util.Set; import java.util.TreeMap; +import java.util.TreeSet; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.LinkedBlockingQueue; @@ -67,16 +69,17 @@ import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.filter.RowFilterInterface; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.io.HbaseMapWritable; +import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.ipc.HMasterRegionInterface; import org.apache.hadoop.hbase.ipc.HRegionInterface; import org.apache.hadoop.hbase.ipc.HbaseRPC; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.InfoServer; import org.apache.hadoop.hbase.util.Sleeper; import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.MapWritable; import org.apache.hadoop.io.Writable; import org.apache.hadoop.ipc.Server; import org.apache.hadoop.net.DNS; @@ -95,9 +98,9 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { // of HRegionServer in isolation. We use AtomicBoolean rather than // plain boolean so we can pass a reference to Chore threads. Otherwise, // Chore threads need to know about the hosting class. - protected volatile AtomicBoolean stopRequested = new AtomicBoolean(false); + protected final AtomicBoolean stopRequested = new AtomicBoolean(false); - protected volatile AtomicBoolean quiesced = new AtomicBoolean(false); + protected final AtomicBoolean quiesced = new AtomicBoolean(false); // Go down hard. Used if file system becomes unavailable and also in // debugging and unit tests. @@ -111,13 +114,15 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { private FileSystem fs; private Path rootDir; private final Random rand = new Random(); - - // region name -> HRegion - protected volatile Map onlineRegions = - new ConcurrentHashMap(); - + + // Key is Bytes.hashCode of region name byte array and the value is HRegion + // in both of the maps below. Use Bytes.mapKey(byte []) generating key for + // below maps. + protected final Map onlineRegions = + new ConcurrentHashMap(); + protected final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - private volatile List outboundMsgs = + private final List outboundMsgs = Collections.synchronizedList(new ArrayList()); final int numRetries; @@ -173,7 +178,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { instance.join(); LOG.info("Shutdown thread complete"); } - } // Compactions @@ -285,7 +289,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { this.outboundMsgs.toArray(new HMsg[outboundMsgs.size()]); this.outboundMsgs.clear(); } - try { this.serverInfo.setLoad(new HServerLoad(requestCount.get(), onlineRegions.size())); @@ -293,12 +296,11 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { HMsg msgs[] = hbaseMaster.regionServerReport( serverInfo, outboundArray, getMostLoadedRegions()); lastMsg = System.currentTimeMillis(); - if (this.quiesced.get() && onlineRegions.size() == 0) { // We've just told the master we're exiting because we aren't // serving any regions. So set the stop bit and exit. LOG.info("Server quiesced and not serving any regions. " + - "Starting shutdown"); + "Starting shutdown"); stopRequested.set(true); this.outboundMsgs.clear(); continue; @@ -310,10 +312,9 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { !restart && !stopRequested.get() && i < msgs.length; i++) { + LOG.info(msgs[i].toString()); switch(msgs[i].getMsg()) { - case HMsg.MSG_CALL_SERVER_STARTUP: - LOG.info("Got call server startup message"); // We the MSG_CALL_SERVER_STARTUP on startup but we can also // get it when the master is panicing because for instance // the HDFS has been yanked out from under it. Be wary of @@ -344,13 +345,11 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { break; case HMsg.MSG_REGIONSERVER_STOP: - LOG.info("Got regionserver stop message"); stopRequested.set(true); break; case HMsg.MSG_REGIONSERVER_QUIESCE: if (!quiesceRequested) { - LOG.info("Got quiesce server message"); try { toDo.put(new ToDoEntry(msgs[i])); } catch (InterruptedException e) { @@ -449,7 +448,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { exitMsg[0] = new HMsg(HMsg.MSG_REPORT_EXITING); // Tell the master what regions we are/were serving int i = 1; - for(HRegion region: closedRegions) { + for (HRegion region: closedRegions) { exitMsg[i++] = new HMsg(HMsg.MSG_REPORT_CLOSE, region.getRegionInfo()); } @@ -472,7 +471,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { * Run init. Sets up hlog and starts up all server threads. * @param c Extra configuration. */ - private void init(final HbaseMapWritable c) throws IOException { + private void init(final MapWritable c) throws IOException { try { for (Map.Entry e: c.entrySet()) { String key = e.getKey().toString(); @@ -676,7 +675,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { * Let the master know we're here * Run initialization using parameters passed us by the master. */ - private HbaseMapWritable reportForDuty(final Sleeper sleeper) + private MapWritable reportForDuty(final Sleeper sleeper) throws IOException { if (LOG.isDebugEnabled()) { LOG.debug("Telling master at " + @@ -687,7 +686,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { HMasterRegionInterface.class, HMasterRegionInterface.versionID, new HServerAddress(conf.get(MASTER_ADDRESS)).getInetSocketAddress(), this.conf); - HbaseMapWritable result = null; + MapWritable result = null; long lastMsg = 0; while(!stopRequested.get()) { try { @@ -695,9 +694,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { this.serverInfo.setLoad(new HServerLoad(0, onlineRegions.size())); lastMsg = System.currentTimeMillis(); result = this.hbaseMaster.regionServerStartup(serverInfo); - if (LOG.isDebugEnabled()) { - LOG.debug("Done telling master we are up"); - } break; } catch (Leases.LeaseStillHeldException e) { LOG.info("Lease " + e.getName() + " already held on master. Check " + @@ -777,7 +773,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { if(e == null || stopRequested.get()) { continue; } - LOG.info(e.msg.toString()); + LOG.info(e.msg); switch(e.msg.getMsg()) { case HMsg.MSG_REGIONSERVER_QUIESCE: @@ -837,13 +833,12 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { } void openRegion(final HRegionInfo regionInfo) { - HRegion region = onlineRegions.get(regionInfo.getRegionName()); + Integer mapKey = Bytes.mapKey(regionInfo.getRegionName()); + HRegion region = this.onlineRegions.get(mapKey); if(region == null) { try { - region = new HRegion( - HTableDescriptor.getTableDir(rootDir, - regionInfo.getTableDesc().getName() - ), + region = new HRegion(HTableDescriptor.getTableDir(rootDir, + regionInfo.getTableDesc().getName()), this.log, this.fs, conf, regionInfo, null, this.cacheFlusher, new Progressable() { public void progress() { @@ -854,8 +849,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { // Startup a compaction early if one is needed. this.compactSplitThread.compactionRequested(region); } catch (IOException e) { - LOG.error("error opening region " + regionInfo.getRegionName(), e); - + LOG.error("error opening region " + regionInfo.getRegionNameAsString(), e); + // TODO: add an extra field in HRegionInfo to indicate that there is // an error. We can't do that now because that would be an incompatible // change that would require a migration @@ -865,7 +860,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { this.lock.writeLock().lock(); try { this.log.setSequenceNumber(region.getMinSequenceId()); - this.onlineRegions.put(region.getRegionName(), region); + this.onlineRegions.put(mapKey, region); } finally { this.lock.writeLock().unlock(); } @@ -889,7 +884,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { this.lock.writeLock().lock(); HRegion region = null; try { - region = onlineRegions.remove(hri.getRegionName()); + region = onlineRegions.remove(Bytes.mapKey(hri.getRegionName())); } finally { this.lock.writeLock().unlock(); } @@ -914,12 +909,13 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { } for(HRegion region: regionsToClose) { if (LOG.isDebugEnabled()) { - LOG.debug("closing region " + region.getRegionName()); + LOG.debug("closing region " + Bytes.toString(region.getRegionName())); } try { region.close(abortRequested, null); } catch (IOException e) { - LOG.error("error closing region " + region.getRegionName(), + LOG.error("error closing region " + + Bytes.toString(region.getRegionName()), RemoteExceptionHandler.checkIOException(e)); } } @@ -932,10 +928,9 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { this.lock.writeLock().lock(); try { synchronized (onlineRegions) { - for (Iterator> i = - onlineRegions.entrySet().iterator(); - i.hasNext();) { - Map.Entry e = i.next(); + for (Iterator> i = + onlineRegions.entrySet().iterator(); i.hasNext();) { + Map.Entry e = i.next(); HRegion r = e.getValue(); if (!r.getRegionInfo().isMetaRegion()) { regionsToClose.add(r); @@ -970,7 +965,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { // /** {@inheritDoc} */ - public HRegionInfo getRegionInfo(final Text regionName) + public HRegionInfo getRegionInfo(final byte [] regionName) throws NotServingRegionException { requestCount.incrementAndGet(); @@ -978,7 +973,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { } /** {@inheritDoc} */ - public Cell get(final Text regionName, final Text row, final Text column) + public Cell get(final byte [] regionName, final byte [] row, + final byte [] column) throws IOException { checkOpen(); requestCount.incrementAndGet(); @@ -991,8 +987,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { } /** {@inheritDoc} */ - public Cell[] get(final Text regionName, final Text row, - final Text column, final int numVersions) + public Cell[] get(final byte [] regionName, final byte [] row, + final byte [] column, final int numVersions) throws IOException { checkOpen(); requestCount.incrementAndGet(); @@ -1005,8 +1001,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { } /** {@inheritDoc} */ - public Cell[] get(final Text regionName, final Text row, final Text column, - final long timestamp, final int numVersions) + public Cell[] get(final byte [] regionName, final byte [] row, + final byte [] column, final long timestamp, final int numVersions) throws IOException { checkOpen(); requestCount.incrementAndGet(); @@ -1019,35 +1015,37 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { } /** {@inheritDoc} */ - public RowResult getRow(final Text regionName, final Text row, final long ts) + public RowResult getRow(final byte [] regionName, final byte [] row, + final long ts) throws IOException { return getRow(regionName, row, null, ts); } /** {@inheritDoc} */ - public RowResult getRow(final Text regionName, final Text row, - final Text[] columns) + public RowResult getRow(final byte [] regionName, final byte [] row, + final byte [][] columns) throws IOException { return getRow(regionName, row, columns, HConstants.LATEST_TIMESTAMP); } /** {@inheritDoc} */ - public RowResult getRow(final Text regionName, final Text row, - final Text[] columns, final long ts) + public RowResult getRow(final byte [] regionName, final byte [] row, + final byte [][] columns, final long ts) throws IOException { checkOpen(); requestCount.incrementAndGet(); try { // convert the columns array into a set so it's easy to check later. - Set columnSet = null; + Set columnSet = null; if (columns != null) { - columnSet = new HashSet(); + columnSet = new TreeSet(Bytes.BYTES_COMPARATOR); columnSet.addAll(Arrays.asList(columns)); } HRegion region = getRegion(regionName); - Map map = region.getFull(row, columnSet, ts); - HbaseMapWritable result = new HbaseMapWritable(); + Map map = region.getFull(row, columnSet, ts); + HbaseMapWritable result = + new HbaseMapWritable(); result.putAll(map); return new RowResult(row, result); } catch (IOException e) { @@ -1057,17 +1055,17 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { } /** {@inheritDoc} */ - public RowResult getClosestRowBefore(final Text regionName, - final Text row) + public RowResult getClosestRowBefore(final byte [] regionName, + final byte [] row) throws IOException { checkOpen(); requestCount.incrementAndGet(); try { // locate the region we're operating on HRegion region = getRegion(regionName); - // ask the region for all the data - return region.getClosestRowBefore(row); + RowResult rr = region.getClosestRowBefore(row); + return rr; } catch (IOException e) { checkFileSystem(); throw e; @@ -1087,15 +1085,16 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { this.leases.renewLease(scannerName); // Collect values to be returned here - HbaseMapWritable values = new HbaseMapWritable(); + HbaseMapWritable values + = new HbaseMapWritable(); HStoreKey key = new HStoreKey(); - TreeMap results = new TreeMap(); + TreeMap results = + new TreeMap(Bytes.BYTES_COMPARATOR); while (s.next(key, results)) { - for(Map.Entry e: results.entrySet()) { + for (Map.Entry e: results.entrySet()) { values.put(e.getKey(), new Cell(e.getValue(), key.getTimestamp())); } - - if(values.size() > 0) { + if (values.size() > 0) { // Row has something in it. Return the value. break; } @@ -1111,8 +1110,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { } /** {@inheritDoc} */ - public void batchUpdate(Text regionName, BatchUpdate b) - throws IOException { + public void batchUpdate(final byte [] regionName, BatchUpdate b) + throws IOException { checkOpen(); this.requestCount.incrementAndGet(); HRegion region = getRegion(regionName); @@ -1130,7 +1129,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { // /** {@inheritDoc} */ - public long openScanner(Text regionName, Text[] cols, Text firstRow, + public long openScanner(byte [] regionName, byte [][] cols, byte [] firstRow, final long timestamp, final RowFilterInterface filter) throws IOException { checkOpen(); @@ -1226,15 +1225,15 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { // /** {@inheritDoc} */ - public void deleteAll(final Text regionName, final Text row, - final Text column, final long timestamp) + public void deleteAll(final byte [] regionName, final byte [] row, + final byte [] column, final long timestamp) throws IOException { HRegion region = getRegion(regionName); region.deleteAll(row, column, timestamp); } /** {@inheritDoc} */ - public void deleteAll(final Text regionName, final Text row, + public void deleteAll(final byte [] regionName, final byte [] row, final long timestamp) throws IOException { HRegion region = getRegion(regionName); @@ -1242,7 +1241,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { } /** {@inheritDoc} */ - public void deleteFamily(Text regionName, Text row, Text family, + public void deleteFamily(byte [] regionName, byte [] row, byte [] family, long timestamp) throws IOException{ getRegion(regionName).deleteFamily(row, family, timestamp); } @@ -1283,8 +1282,17 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { /** * @return Immutable list of this servers regions. */ - public Map getOnlineRegions() { - return Collections.unmodifiableMap(onlineRegions); + public Collection getOnlineRegions() { + return Collections.unmodifiableCollection(onlineRegions.values()); + } + + /** + * @param regionName + * @return HRegion for the passed regionName or null if named + * region is not member of the online regions. + */ + public HRegion getOnlineRegion(final byte [] regionName) { + return onlineRegions.get(Bytes.mapKey(regionName)); } /** @return the request count */ @@ -1303,17 +1311,16 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { * @return {@link HRegion} for regionName * @throws NotServingRegionException */ - protected HRegion getRegion(final Text regionName) + protected HRegion getRegion(final byte [] regionName) throws NotServingRegionException { HRegion region = null; this.lock.readLock().lock(); try { - region = onlineRegions.get(regionName); - + Integer key = Integer.valueOf(Bytes.hashCode(regionName)); + region = onlineRegions.get(key); if (region == null) { - throw new NotServingRegionException(regionName.toString()); + throw new NotServingRegionException(regionName); } - return region; } finally { this.lock.readLock().unlock(); @@ -1329,9 +1336,9 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { protected HRegionInfo[] getMostLoadedRegions() { ArrayList regions = new ArrayList(); synchronized (onlineRegions) { - for (Map.Entry entry : onlineRegions.entrySet()) { + for (HRegion r : onlineRegions.values()) { if (regions.size() < numRegionsToReport) { - regions.add(entry.getValue().getRegionInfo()); + regions.add(r.getRegionInfo()); } else { break; } diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HStore.java b/src/java/org/apache/hadoop/hbase/regionserver/HStore.java index d113cdb4bb6..8fb8057c366 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -52,11 +52,10 @@ import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.filter.RowFilterInterface; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.hadoop.hbase.io.TextSequence; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.io.MapFile; import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.StringUtils; @@ -104,7 +103,8 @@ public class HStore implements HConstants { private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - final Text storeName; + final byte [] storeName; + private final String storeNameStr; /* * Sorted Map of readers keyed by sequence id (Most recent should be last in @@ -180,8 +180,9 @@ public class HStore implements HConstants { this.ttl *= 1000; this.memcache = new Memcache(this.ttl); this.compactionDir = HRegion.getCompactionDir(basedir); - this.storeName = - new Text(this.info.getEncodedName() + "/" + this.family.getFamilyName()); + this.storeName = Bytes.toBytes(this.info.getEncodedName() + "/" + + Bytes.toString(this.family.getName())); + this.storeNameStr = Bytes.toString(this.storeName); // By default, we compact if an HStore has more than // MIN_COMMITS_FOR_COMPACTION map files @@ -203,12 +204,12 @@ public class HStore implements HConstants { } Path mapdir = HStoreFile.getMapDir(basedir, info.getEncodedName(), - family.getFamilyName()); + family.getName()); if (!fs.exists(mapdir)) { fs.mkdirs(mapdir); } Path infodir = HStoreFile.getInfoDir(basedir, info.getEncodedName(), - family.getFamilyName()); + family.getName()); if (!fs.exists(infodir)) { fs.mkdirs(infodir); } @@ -218,7 +219,7 @@ public class HStore implements HConstants { this.bloomFilter = null; } else { this.filterDir = HStoreFile.getFilterDir(basedir, info.getEncodedName(), - family.getFamilyName()); + family.getName()); if (!fs.exists(filterDir)) { fs.mkdirs(filterDir); } @@ -232,9 +233,9 @@ public class HStore implements HConstants { // loadHStoreFiles also computes the max sequence id internally. this.maxSeqId = -1L; this.storefiles.putAll(loadHStoreFiles(infodir, mapdir)); - if (LOG.isDebugEnabled()) { + if (LOG.isDebugEnabled() && this.storefiles.size() > 0) { LOG.debug("Loaded " + this.storefiles.size() + " file(s) in hstore " + - this.storeName + ", max sequence id " + this.maxSeqId); + Bytes.toString(this.storeName) + ", max sequence id " + this.maxSeqId); } try { @@ -244,7 +245,7 @@ public class HStore implements HConstants { // HADOOP-1700; for now keep going but this is probably not what we want // long term. If we got here there has been data-loss LOG.warn("Exception processing reconstruction log " + reconstructionLog + - " opening " + this.storeName + + " opening " + Bytes.toString(this.storeName) + " -- continuing. Probably DATA LOSS!", e); } @@ -317,10 +318,10 @@ public class HStore implements HConstants { } // Check this edit is for me. Also, guard against writing // METACOLUMN info such as HBASE::CACHEFLUSH entries - Text column = val.getColumn(); - if (column.equals(HLog.METACOLUMN) - || !key.getRegionName().equals(info.getRegionName()) - || !HStoreKey.extractFamily(column).equals(family.getFamilyName())) { + byte [] column = val.getColumn(); + if (Bytes.equals(column, HLog.METACOLUMN) + || !Bytes.equals(key.getRegionName(), info.getRegionName()) + || !HStoreKey.matchingFamily(family.getName(), column)) { continue; } HStoreKey k = new HStoreKey(key.getRow(), column, val.getTimestamp()); @@ -386,7 +387,7 @@ public class HStore implements HConstants { reference = readSplitInfo(p, fs); } curfile = new HStoreFile(conf, fs, basedir, info.getEncodedName(), - family.getFamilyName(), fid, reference); + family.getName(), fid, reference); storeSize += curfile.length(); long storeSeqId = -1; try { @@ -450,7 +451,7 @@ public class HStore implements HConstants { Filter bloomFilter = null; if(fs.exists(filterFile)) { if (LOG.isDebugEnabled()) { - LOG.debug("loading bloom filter for " + this.storeName); + LOG.debug("loading bloom filter for " + this.storeNameStr); } BloomFilterDescriptor.BloomFilterType type = @@ -482,7 +483,7 @@ public class HStore implements HConstants { } } else { if (LOG.isDebugEnabled()) { - LOG.debug("creating bloom filter for " + this.storeName); + LOG.debug("creating bloom filter for " + this.storeNameStr); } BloomFilterDescriptor.BloomFilterType type = @@ -517,7 +518,7 @@ public class HStore implements HConstants { */ private void flushBloomFilter() throws IOException { if (LOG.isDebugEnabled()) { - LOG.debug("flushing bloom filter for " + this.storeName); + LOG.debug("flushing bloom filter for " + this.storeNameStr); } FSDataOutputStream out = fs.create(new Path(filterDir, BLOOMFILTER_FILE_NAME)); @@ -527,7 +528,7 @@ public class HStore implements HConstants { out.close(); } if (LOG.isDebugEnabled()) { - LOG.debug("flushed bloom filter for " + this.storeName); + LOG.debug("flushed bloom filter for " + this.storeNameStr); } } @@ -568,7 +569,7 @@ public class HStore implements HConstants { synchronized (this.storefiles) { result = new ArrayList(storefiles.values()); } - LOG.debug("closed " + this.storeName); + LOG.debug("closed " + this.storeNameStr); return result; } finally { this.lock.writeLock().unlock(); @@ -621,7 +622,7 @@ public class HStore implements HConstants { long now = System.currentTimeMillis(); // A. Write the Maps out to the disk HStoreFile flushedFile = new HStoreFile(conf, fs, basedir, - info.getEncodedName(), family.getFamilyName(), -1L, null); + info.getEncodedName(), family.getName(), -1L, null); MapFile.Writer out = flushedFile.getWriter(this.fs, this.compression, this.bloomFilter); @@ -641,8 +642,7 @@ public class HStore implements HConstants { for (Map.Entry es: cache.entrySet()) { HStoreKey curkey = es.getKey(); byte[] bytes = es.getValue(); - TextSequence f = HStoreKey.extractFamily(curkey.getColumn()); - if (f.equals(this.family.getFamilyName())) { + if (HStoreKey.matchingFamily(this.family.getName(), curkey.getColumn())) { if (ttl == HConstants.FOREVER || now < curkey.getTimestamp() + ttl) { entries++; @@ -775,11 +775,14 @@ public class HStore implements HConstants { * @return mid key if a split is needed, null otherwise * @throws IOException */ - Text compact(final boolean force) throws IOException { + byte [] compact(final boolean force) throws IOException { synchronized (compactLock) { long maxId = -1; List filesToCompact = null; synchronized (storefiles) { + if (this.storefiles.size() <= 0) { + return null; + } filesToCompact = new ArrayList(this.storefiles.values()); if (!force && !hasReferences(filesToCompact) && filesToCompact.size() < compactionThreshold) { @@ -800,7 +803,7 @@ public class HStore implements HConstants { // Step through them, writing to the brand-new MapFile HStoreFile compactedOutputFile = new HStoreFile(conf, fs, - this.compactionDir, info.getEncodedName(), family.getFamilyName(), + this.compactionDir, info.getEncodedName(), family.getName(), -1L, null); if (LOG.isDebugEnabled()) { LOG.debug("started compaction of " + filesToCompact.size() + @@ -822,7 +825,7 @@ public class HStore implements HConstants { completeCompaction(filesToCompact, compactedOutputFile); if (LOG.isDebugEnabled()) { - LOG.debug("Completed compaction of " + this.storeName + + LOG.debug("Completed compaction of " + this.storeNameStr + " store size is " + StringUtils.humanReadableInt(storeSize)); } } @@ -883,10 +886,10 @@ public class HStore implements HConstants { long now = System.currentTimeMillis(); int timesSeen = 0; - Text lastRow = new Text(); - Text lastColumn = new Text(); + byte [] lastRow = null; + byte [] lastColumn = null; // Map of a row deletes keyed by column with a list of timestamps for value - Map> deletes = null; + Map> deletes = null; while (numDone < done.length) { // Find the reader with the smallest key. If two files have same key // but different values -- i.e. one is delete and other is non-delete @@ -909,13 +912,13 @@ public class HStore implements HConstants { // Reflect the current key/val in the output HStoreKey sk = keys[smallestKey]; - if(lastRow.equals(sk.getRow()) - && lastColumn.equals(sk.getColumn())) { + if (Bytes.equals(lastRow, sk.getRow()) + && Bytes.equals(lastColumn, sk.getColumn())) { timesSeen++; } else { - timesSeen = 1; + timesSeen = 0; // We are on to a new row. Create a new deletes list. - deletes = new HashMap>(); + deletes = new TreeMap>(Bytes.BYTES_COMPARATOR); } byte [] value = (vals[smallestKey] == null)? @@ -924,7 +927,7 @@ public class HStore implements HConstants { timesSeen <= family.getMaxVersions()) { // Keep old versions until we have maxVersions worth. // Then just skip them. - if (sk.getRow().getLength() != 0 && sk.getColumn().getLength() != 0) { + if (sk.getRow().length != 0 && sk.getColumn().length != 0) { // Only write out objects which have a non-zero length key and // value if (ttl == HConstants.FOREVER || now < sk.getTimestamp() + ttl) { @@ -938,8 +941,8 @@ public class HStore implements HConstants { } // Update last-seen items - lastRow.set(sk.getRow()); - lastColumn.set(sk.getColumn()); + lastRow = sk.getRow(); + lastColumn = sk.getColumn(); // Advance the smallest key. If that reader's all finished, then // mark it as done. @@ -962,7 +965,7 @@ public class HStore implements HConstants { try { rdrs[i].close(); } catch (IOException e) { - LOG.warn("Exception closing reader for " + this.storeName, e); + LOG.warn("Exception closing reader for " + this.storeNameStr, e); } } } @@ -984,7 +987,7 @@ public class HStore implements HConstants { * passed value is HGlobals.deleteBytes. */ private boolean isDeleted(final HStoreKey hsk, final byte [] value, - final boolean checkMemcache, final Map> deletes) { + final boolean checkMemcache, final Map> deletes) { if (checkMemcache && memcache.isDeleted(hsk)) { return true; } @@ -1042,7 +1045,7 @@ public class HStore implements HConstants { try { // 1. Moving the new MapFile into place. HStoreFile finalCompactedFile = new HStoreFile(conf, fs, basedir, - info.getEncodedName(), family.getFamilyName(), -1, null); + info.getEncodedName(), family.getName(), -1, null); if (LOG.isDebugEnabled()) { LOG.debug("moving " + FSUtils.getPath(compactedFile.getMapFilePath()) + " to " + FSUtils.getPath(finalCompactedFile.getMapFilePath())); @@ -1093,7 +1096,8 @@ public class HStore implements HConstants { } } catch (IOException e) { e = RemoteExceptionHandler.checkIOException(e); - LOG.error("Failed replacing compacted files for " + this.storeName + + LOG.error("Failed replacing compacted files for " + + this.storeNameStr + ". Compacted file is " + finalCompactedFile.toString() + ". Files replaced are " + compactedFiles.toString() + " some of which may have been already removed", e); @@ -1120,10 +1124,12 @@ public class HStore implements HConstants { * * The returned object should map column names to Cells. */ - void getFull(HStoreKey key, final Set columns, Map results) + void getFull(HStoreKey key, final Set columns, + Map results) throws IOException { - Map deletes = new HashMap(); - + Map deletes = + new TreeMap(Bytes.BYTES_COMPARATOR); + // if the key is null, we're not even looking for anything. return. if (key == null) { return; @@ -1152,7 +1158,7 @@ public class HStore implements HConstants { } private void getFullFromMapFile(MapFile.Reader map, HStoreKey key, - Set columns, Map deletes, Map results) + Set columns, Map deletes, Map results) throws IOException { synchronized(map) { long now = System.currentTimeMillis(); @@ -1167,7 +1173,7 @@ public class HStore implements HConstants { return; } do { - Text readcol = readkey.getColumn(); + byte [] readcol = readkey.getColumn(); // if we're looking for this column (or all of them), and there isn't // already a value for this column in the results map, and the key we @@ -1182,18 +1188,18 @@ public class HStore implements HConstants { // recent delete timestamp, record it for later if (!deletes.containsKey(readcol) || deletes.get(readcol).longValue() < readkey.getTimestamp()) { - deletes.put(new Text(readcol), readkey.getTimestamp()); + deletes.put(readcol, readkey.getTimestamp()); } } else if (!(deletes.containsKey(readcol) && deletes.get(readcol).longValue() >= readkey.getTimestamp()) ) { // So the cell itself isn't a delete, but there may be a delete // pending from earlier in our search. Only record this result if // there aren't any pending deletes. - if (!(deletes.containsKey(readcol) - && deletes.get(readcol).longValue() >= readkey.getTimestamp())) { + if (!(deletes.containsKey(readcol) && + deletes.get(readcol).longValue() >= readkey.getTimestamp())) { if (ttl == HConstants.FOREVER || now < readkey.getTimestamp() + ttl) { - results.put(new Text(readcol), + results.put(readcol, new Cell(readval.get(), readkey.getTimestamp())); // need to reinstantiate the readval so we can reuse it, // otherwise next iteration will destroy our result @@ -1205,7 +1211,7 @@ public class HStore implements HConstants { } } } - } else if(key.getRow().compareTo(readkey.getRow()) < 0) { + } else if (Bytes.compareTo(key.getRow(), readkey.getRow()) < 0) { // if we've crossed into the next row, then we can just stop // iterating break; @@ -1253,7 +1259,8 @@ public class HStore implements HConstants { // This List of deletes should not large since we are only keeping rows // and columns that match those set on the scanner and which have delete // values. If memory usage becomes an issue, could redo as bloom filter. - Map> deletes = new HashMap>(); + Map> deletes = + new TreeMap>(Bytes.BYTES_COMPARATOR); // This code below is very close to the body of the getKeys method. MapFile.Reader[] maparray = getReaders(); for(int i = maparray.length - 1; i >= 0; i--) { @@ -1337,8 +1344,7 @@ public class HStore implements HConstants { * @throws IOException */ List getKeys(final HStoreKey origin, final int versions) - throws IOException { - + throws IOException { List keys = this.memcache.getKeys(origin, versions); if (versions != ALL_VERSIONS && keys.size() >= versions) { return keys; @@ -1414,13 +1420,13 @@ public class HStore implements HConstants { * with stricly increasing timestamps. This method assumes this pattern of * writes in order to make it reasonably performant. */ - Text getRowKeyAtOrBefore(final Text row) + byte [] getRowKeyAtOrBefore(final byte [] row) throws IOException{ // Map of HStoreKeys that are candidates for holding the row key that // most closely matches what we're looking for. We'll have to update it // deletes found all over the place as we go along before finally reading // the best key out of it at the end. - SortedMap candidateKeys = new TreeMap(); + SortedMap candidateKeys = new TreeMap(); // Obtain read lock this.lock.readLock().lock(); @@ -1446,14 +1452,13 @@ public class HStore implements HConstants { * Check an individual MapFile for the row at or before a given key * and timestamp */ - private void rowAtOrBeforeFromMapFile(MapFile.Reader map, Text row, + private void rowAtOrBeforeFromMapFile(MapFile.Reader map, final byte [] row, SortedMap candidateKeys) throws IOException { HStoreKey searchKey = null; ImmutableBytesWritable readval = new ImmutableBytesWritable(); HStoreKey readkey = new HStoreKey(); HStoreKey strippedKey = null; - synchronized(map) { // don't bother with the rest of this if the file is empty map.reset(); @@ -1462,7 +1467,6 @@ public class HStore implements HConstants { } long now = System.currentTimeMillis(); - // if there aren't any candidate keys yet, we'll do some things slightly // different if (candidateKeys.isEmpty()) { @@ -1472,7 +1476,7 @@ public class HStore implements HConstants { // save time and add the last key to the candidates. HStoreKey finalKey = new HStoreKey(); map.finalKey(finalKey); - if (finalKey.getRow().compareTo(row) < 0) { + if (Bytes.compareTo(finalKey.getRow(), row) < 0) { candidateKeys.put(stripTimestamp(finalKey), new Long(finalKey.getTimestamp())); return; @@ -1489,7 +1493,7 @@ public class HStore implements HConstants { do { // if we have an exact match on row, and it's not a delete, save this // as a candidate key - if (readkey.getRow().equals(row)) { + if (Bytes.equals(readkey.getRow(), row)) { if (!HLogEdit.isDeleted(readval.get())) { if (ttl == HConstants.FOREVER || now < readkey.getTimestamp() + ttl) { @@ -1502,7 +1506,7 @@ public class HStore implements HConstants { } } } - } else if (readkey.getRow().compareTo(row) > 0 ) { + } else if (Bytes.compareTo(readkey.getRow(), row) > 0 ) { // if the row key we just read is beyond the key we're searching for, // then we're done. return. return; @@ -1538,7 +1542,7 @@ public class HStore implements HConstants { // save time and add the last key to the candidates. HStoreKey finalKey = new HStoreKey(); map.finalKey(finalKey); - if (finalKey.getRow().compareTo(searchKey.getRow()) < 0) { + if (Bytes.compareTo(finalKey.getRow(), searchKey.getRow()) < 0) { strippedKey = stripTimestamp(finalKey); // if the candidate keys has a cell like this one already, @@ -1568,7 +1572,7 @@ public class HStore implements HConstants { do { // if we have an exact match on row, and it's not a delete, save this // as a candidate key - if (readkey.getRow().equals(row)) { + if (Bytes.equals(readkey.getRow(), row)) { strippedKey = stripTimestamp(readkey); if (!HLogEdit.isDeleted(readval.get())) { if (ttl == HConstants.FOREVER || @@ -1593,7 +1597,7 @@ public class HStore implements HConstants { } } } - } else if (readkey.getRow().compareTo(row) > 0 ) { + } else if (Bytes.compareTo(readkey.getRow(), row) > 0 ) { // if the row key we just read is beyond the key we're searching for, // then we're done. return. return; @@ -1644,9 +1648,9 @@ public class HStore implements HConstants { */ private boolean cellMatches(HStoreKey origin, HStoreKey target){ // if the origin's column is empty, then we're matching any column - if (origin.getColumn().equals(new Text())){ + if (Bytes.equals(origin.getColumn(), HConstants.EMPTY_BYTE_ARRAY)){ // if the row matches, then... - if (target.getRow().equals(origin.getRow())) { + if (Bytes.equals(target.getRow(), origin.getRow())) { // check the timestamp return target.getTimestamp() <= origin.getTimestamp(); } @@ -1665,9 +1669,9 @@ public class HStore implements HConstants { */ private boolean rowMatches(HStoreKey origin, HStoreKey target){ // if the origin's column is empty, then we're matching any column - if (origin.getColumn().equals(new Text())){ + if (Bytes.equals(origin.getColumn(), HConstants.EMPTY_BYTE_ARRAY)) { // if the row matches, then... - return target.getRow().equals(origin.getRow()); + return Bytes.equals(target.getRow(), origin.getRow()); } // otherwise, we want to match on row and column return target.matchesRowCol(origin); @@ -1678,7 +1682,7 @@ public class HStore implements HConstants { * * @return midKey if store can be split, null otherwise */ - Text checkSplit() { + byte [] checkSplit() { if (this.storefiles.size() <= 0) { return null; } @@ -1726,14 +1730,14 @@ public class HStore implements HConstants { if (mk != null) { // if the midkey is the same as the first and last keys, then we cannot // (ever) split this region. - if (mk.getRow().equals(firstKey.getRow()) && - mk.getRow().equals(lastKey.getRow())) { + if (Bytes.equals(mk.getRow(), firstKey.getRow()) && + Bytes.equals(mk.getRow(), lastKey.getRow())) { return null; } return mk.getRow(); } } catch(IOException e) { - LOG.warn("Failed getting store size for " + this.storeName, e); + LOG.warn("Failed getting store size for " + this.storeNameStr, e); } finally { this.lock.readLock().unlock(); } @@ -1752,8 +1756,8 @@ public class HStore implements HConstants { /** * Return a scanner for both the memcache and the HStore files */ - InternalScanner getScanner(long timestamp, Text targetCols[], - Text firstRow, RowFilterInterface filter) + InternalScanner getScanner(long timestamp, byte [][] targetCols, + byte [] firstRow, RowFilterInterface filter) throws IOException { lock.readLock().lock(); try { @@ -1766,7 +1770,7 @@ public class HStore implements HConstants { /** {@inheritDoc} */ @Override public String toString() { - return this.storeName.toString(); + return this.storeNameStr; } /* diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java b/src/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java index c5493b9ef5c..b2a6828e0c6 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java @@ -39,10 +39,10 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.io.BlockFSInputStream; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.io.MapFile; import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableComparable; import org.onelab.filter.Filter; @@ -50,6 +50,7 @@ import org.onelab.filter.Key; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HStoreKey; /** @@ -113,8 +114,8 @@ public class HStoreFile implements HConstants { private final static Random rand = new Random(); private final Path basedir; - private final String encodedRegionName; - private final Text colFamily; + private final int encodedRegionName; + private final byte [] colFamily; private final long fileId; private final HBaseConfiguration conf; private final FileSystem fs; @@ -131,13 +132,13 @@ public class HStoreFile implements HConstants { * @throws IOException */ HStoreFile(HBaseConfiguration conf, FileSystem fs, Path basedir, - String encodedRegionName, Text colFamily, long fileId, + int encodedRegionName, byte [] colFamily, long fileId, final Reference ref) throws IOException { this.conf = conf; this.fs = fs; this.basedir = basedir; this.encodedRegionName = encodedRegionName; - this.colFamily = new Text(colFamily); + this.colFamily = colFamily; long id = fileId; if (id == -1) { @@ -145,7 +146,7 @@ public class HStoreFile implements HConstants { Path testpath = null; do { id = Math.abs(rand.nextLong()); - testpath = new Path(mapdir, createHStoreFilename(id, null)); + testpath = new Path(mapdir, createHStoreFilename(id, -1)); } while(fs.exists(testpath)); } this.fileId = id; @@ -165,12 +166,12 @@ public class HStoreFile implements HConstants { return reference; } - String getEncodedRegionName() { - return encodedRegionName; + int getEncodedRegionName() { + return this.encodedRegionName; } /** @return the column family */ - Text getColFamily() { + byte [] getColFamily() { return colFamily; } @@ -187,18 +188,22 @@ public class HStoreFile implements HConstants { return getMapFilePath(encodedRegionName, fileId, reference.getEncodedRegionName()); } - return getMapFilePath(encodedRegionName, fileId, null); + return getMapFilePath(this.encodedRegionName, fileId); } private Path getMapFilePath(final Reference r) { if (r == null) { return getMapFilePath(); } - return getMapFilePath(r.getEncodedRegionName(), r.getFileId(), null); + return getMapFilePath(r.getEncodedRegionName(), r.getFileId()); } - private Path getMapFilePath(final String encodedName, final long fid, - final String ern) { + private Path getMapFilePath(final int encodedName, final long fid) { + return getMapFilePath(encodedName, fid, HRegionInfo.NO_HASH); + } + + private Path getMapFilePath(final int encodedName, final long fid, + final int ern) { return new Path(HStoreFile.getMapDir(basedir, encodedName, colFamily), createHStoreFilename(fid, ern)); } @@ -210,11 +215,15 @@ public class HStoreFile implements HConstants { reference.getEncodedRegionName()); } - return getInfoFilePath(encodedRegionName, fileId, null); + return getInfoFilePath(encodedRegionName, fileId); } - private Path getInfoFilePath(final String encodedName, final long fid, - final String ern) { + private Path getInfoFilePath(final int encodedName, final long fid) { + return getInfoFilePath(encodedName, fid, HRegionInfo.NO_HASH); + } + + private Path getInfoFilePath(final int encodedName, final long fid, + final int ern) { return new Path(HStoreFile.getInfoDir(basedir, encodedName, colFamily), createHStoreFilename(fid, ern)); } @@ -293,8 +302,7 @@ public class HStoreFile implements HConstants { long loadInfo(FileSystem fs) throws IOException { Path p = null; if (isReference()) { - p = getInfoFilePath(reference.getEncodedRegionName(), - reference.getFileId(), null); + p = getInfoFilePath(reference.getEncodedRegionName(), reference.getFileId()); } else { p = getInfoFilePath(); } @@ -400,7 +408,6 @@ public class HStoreFile implements HConstants { public synchronized MapFile.Reader getReader(final FileSystem fs, final Filter bloomFilter, final boolean blockCacheEnabled) throws IOException { - if (isReference()) { return new HStoreFile.HalfMapFileReader(fs, getMapFilePath(reference).toString(), conf, @@ -475,55 +482,72 @@ public class HStoreFile implements HConstants { return r.equals(Range.top); } - private static String createHStoreFilename(final long fid, - final String encodedRegionName) { - return Long.toString(fid) + - ((encodedRegionName != null) ? "." + encodedRegionName : ""); + private static String createHStoreFilename(final long fid) { + return createHStoreFilename(fid, HRegionInfo.NO_HASH); } - + + private static String createHStoreFilename(final long fid, + final int encodedRegionName) { + return Long.toString(fid) + + ((encodedRegionName != HRegionInfo.NO_HASH)? + "." + encodedRegionName : ""); + } + /** - * @param dir - * @param encodedRegionName - * @param colFamily + * @param dir Base directory + * @param encodedRegionName Encoding of region name. + * @param f Column family. * @return path for map file directory */ - public static Path getMapDir(Path dir, String encodedRegionName, Text colFamily) { - return new Path(dir, new Path(encodedRegionName, - new Path(colFamily.toString(), HSTORE_DATFILE_DIR))); + public static Path getMapDir(Path dir, int encodedRegionName, + final byte [] f) { + return getFamilySubDir(dir, encodedRegionName, f, HSTORE_DATFILE_DIR); } /** - * @param dir - * @param encodedRegionName - * @param colFamily + * @param dir Base directory + * @param encodedRegionName Encoding of region name. + * @param f Column family. * @return the info directory path */ - public static Path getInfoDir(Path dir, String encodedRegionName, Text colFamily) { - return new Path(dir, new Path(encodedRegionName, - new Path(colFamily.toString(), HSTORE_INFO_DIR))); + public static Path getInfoDir(Path dir, int encodedRegionName, byte [] f) { + return getFamilySubDir(dir, encodedRegionName, f, HSTORE_INFO_DIR); } /** - * @param dir - * @param encodedRegionName - * @param colFamily + * @param dir Base directory + * @param encodedRegionName Encoding of region name. + * @param f Column family. * @return the bloom filter directory path */ - public static Path getFilterDir(Path dir, String encodedRegionName, Text colFamily) { - return new Path(dir, new Path(encodedRegionName, - new Path(colFamily.toString(), HSTORE_FILTER_DIR))); + public static Path getFilterDir(Path dir, int encodedRegionName, + final byte [] f) { + return getFamilySubDir(dir, encodedRegionName, f, HSTORE_FILTER_DIR); + } + + /* + * @param base Base directory + * @param encodedRegionName Encoding of region name. + * @param f Column family. + * @param subdir Subdirectory to create under column family/store directory. + * @return + */ + private static Path getFamilySubDir(final Path base, + final int encodedRegionName, final byte [] f, final String subdir) { + return new Path(base, new Path(Integer.toString(encodedRegionName), + new Path(Bytes.toString(f), subdir))); } /* * Data structure to hold reference to a store file over in another region. */ static class Reference implements Writable { - private String encodedRegionName; + private int encodedRegionName; private long fileid; private Range region; private HStoreKey midkey; - Reference(final String ern, final long fid, final HStoreKey m, + Reference(final int ern, final long fid, final HStoreKey m, final Range fr) { this.encodedRegionName = ern; this.fileid = fid; @@ -532,7 +556,7 @@ public class HStoreFile implements HConstants { } Reference() { - this(null, -1, null, Range.bottom); + this(-1, -1, null, Range.bottom); } long getFileId() { @@ -547,8 +571,8 @@ public class HStoreFile implements HConstants { return midkey; } - String getEncodedRegionName() { - return encodedRegionName; + int getEncodedRegionName() { + return this.encodedRegionName; } /** {@inheritDoc} */ @@ -561,7 +585,7 @@ public class HStoreFile implements HConstants { /** {@inheritDoc} */ public void write(DataOutput out) throws IOException { - out.writeUTF(encodedRegionName); + out.writeInt(this.encodedRegionName); out.writeLong(fileid); // Write true if we're doing top of the file. out.writeBoolean(isTopFileRegion(region)); @@ -570,7 +594,7 @@ public class HStoreFile implements HConstants { /** {@inheritDoc} */ public void readFields(DataInput in) throws IOException { - encodedRegionName = in.readUTF(); + this.encodedRegionName = in.readInt(); fileid = in.readLong(); boolean tmp = in.readBoolean(); // If true, set region to top. diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HStoreScanner.java b/src/java/org/apache/hadoop/hbase/regionserver/HStoreScanner.java index 017f4dca11a..2ae031da215 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HStoreScanner.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HStoreScanner.java @@ -21,19 +21,19 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; -import java.util.TreeMap; -import java.util.SortedMap; -import java.util.Iterator; -import java.util.Map; -import java.util.List; import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HStoreKey; import org.apache.hadoop.hbase.filter.RowFilterInterface; -import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.util.Bytes; /** * Scanner scans both the memcache and the HStore @@ -42,7 +42,7 @@ class HStoreScanner implements InternalScanner { static final Log LOG = LogFactory.getLog(HStoreScanner.class); private InternalScanner[] scanners; - private TreeMap[] resultSets; + private TreeMap[] resultSets; private HStoreKey[] keys; private boolean wildcardMatch = false; private boolean multipleMatchers = false; @@ -51,8 +51,8 @@ class HStoreScanner implements InternalScanner { /** Create an Scanner with a handle on the memcache and HStore files. */ @SuppressWarnings("unchecked") - HStoreScanner(HStore store, Text[] targetCols, Text firstRow, long timestamp, - RowFilterInterface filter) + HStoreScanner(HStore store, byte [][] targetCols, byte [] firstRow, + long timestamp, RowFilterInterface filter) throws IOException { this.store = store; this.dataFilter = filter; @@ -87,7 +87,7 @@ class HStoreScanner implements InternalScanner { // All results will match the required column-set and scanTime. for (int i = 0; i < scanners.length; i++) { keys[i] = new HStoreKey(); - resultSets[i] = new TreeMap(); + resultSets[i] = new TreeMap(Bytes.BYTES_COMPARATOR); if(scanners[i] != null && !scanners[i].next(keys[i], resultSets[i])) { closeScanner(i); } @@ -105,7 +105,7 @@ class HStoreScanner implements InternalScanner { } /** {@inheritDoc} */ - public boolean next(HStoreKey key, SortedMap results) + public boolean next(HStoreKey key, SortedMap results) throws IOException { // Filtered flag is set by filters. If a cell has been 'filtered out' @@ -114,15 +114,15 @@ class HStoreScanner implements InternalScanner { boolean moreToFollow = true; while (filtered && moreToFollow) { // Find the lowest-possible key. - Text chosenRow = null; + byte [] chosenRow = null; long chosenTimestamp = -1; for (int i = 0; i < this.keys.length; i++) { if (scanners[i] != null && (chosenRow == null || - (keys[i].getRow().compareTo(chosenRow) < 0) || - ((keys[i].getRow().compareTo(chosenRow) == 0) && + (Bytes.compareTo(keys[i].getRow(), chosenRow) < 0) || + ((Bytes.compareTo(keys[i].getRow(), chosenRow) == 0) && (keys[i].getTimestamp() > chosenTimestamp)))) { - chosenRow = new Text(keys[i].getRow()); + chosenRow = keys[i].getRow(); chosenTimestamp = keys[i].getTimestamp(); } } @@ -136,7 +136,7 @@ class HStoreScanner implements InternalScanner { // Here we are setting the passed in key with current row+timestamp key.setRow(chosenRow); key.setVersion(chosenTimestamp); - key.setColumn(HConstants.EMPTY_TEXT); + key.setColumn(HConstants.EMPTY_BYTE_ARRAY); // Keep list of deleted cell keys within this row. We need this // because as we go through scanners, the delete record may be in an // early scanner and then the same record with a non-delete, non-null @@ -150,7 +150,7 @@ class HStoreScanner implements InternalScanner { while ((scanners[i] != null && !filtered && moreToFollow) - && (keys[i].getRow().compareTo(chosenRow) == 0)) { + && (Bytes.compareTo(keys[i].getRow(), chosenRow) == 0)) { // If we are doing a wild card match or there are multiple // matchers per column, we need to scan all the older versions of // this row to pick up the rest of the family members @@ -164,9 +164,9 @@ class HStoreScanner implements InternalScanner { // but this had the effect of overwriting newer // values with older ones. So now we only insert // a result if the map does not contain the key. - HStoreKey hsk = new HStoreKey(key.getRow(), HConstants.EMPTY_TEXT, + HStoreKey hsk = new HStoreKey(key.getRow(), HConstants.EMPTY_BYTE_ARRAY, key.getTimestamp()); - for (Map.Entry e : resultSets[i].entrySet()) { + for (Map.Entry e : resultSets[i].entrySet()) { hsk.setColumn(e.getKey()); if (HLogEdit.isDeleted(e.getValue())) { if (!deletes.contains(hsk)) { @@ -202,7 +202,7 @@ class HStoreScanner implements InternalScanner { // If the current scanner is non-null AND has a lower-or-equal // row label, then its timestamp is bad. We need to advance it. while ((scanners[i] != null) && - (keys[i].getRow().compareTo(chosenRow) <= 0)) { + (Bytes.compareTo(keys[i].getRow(), chosenRow) <= 0)) { resultSets[i].clear(); if (!scanners[i].next(keys[i], resultSets[i])) { closeScanner(i); @@ -266,7 +266,7 @@ class HStoreScanner implements InternalScanner { } } - public Iterator>> iterator() { + public Iterator>> iterator() { throw new UnsupportedOperationException("Unimplemented serverside. " + "next(HStoreKey, StortedMap(...) is more efficient"); } diff --git a/src/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java b/src/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java index f1196c1061e..360bd27f13a 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/InternalScanner.java @@ -22,7 +22,6 @@ package org.apache.hadoop.hbase.regionserver; import java.io.Closeable; import java.io.IOException; import java.util.SortedMap; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.HStoreKey; /** @@ -50,7 +49,7 @@ public interface InternalScanner extends Closeable { * @return true if data was returned * @throws IOException */ - public boolean next(HStoreKey key, SortedMap results) + public boolean next(HStoreKey key, SortedMap results) throws IOException; /** diff --git a/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java b/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java index 3165104286f..b482693952f 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java @@ -24,14 +24,13 @@ import java.io.IOException; import java.rmi.UnexpectedException; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; +import java.util.TreeSet; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.commons.logging.Log; @@ -39,8 +38,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HStoreKey; import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.hbase.util.Bytes; /** @@ -200,25 +198,25 @@ class Memcache { * @return Return lowest of a or b or null if both a and b are null */ @SuppressWarnings("unchecked") - private WritableComparable getLowest(final WritableComparable a, - final WritableComparable b) { + private byte [] getLowest(final byte [] a, + final byte [] b) { if (a == null) { return b; } if (b == null) { return a; } - return a.compareTo(b) <= 0? a: b; + return Bytes.compareTo(a, b) <= 0? a: b; } /** * @param row Find the row that comes after this one. * @return Next row or null if none found */ - Text getNextRow(final Text row) { + byte [] getNextRow(final byte [] row) { this.lock.readLock().lock(); try { - return (Text)getLowest(getNextRow(row, this.memcache), + return getLowest(getNextRow(row, this.memcache), getNextRow(row, this.snapshot)); } finally { this.lock.readLock().unlock(); @@ -231,9 +229,9 @@ class Memcache { * This method synchronizes on passed map while iterating it. * @return Next row or null if none found. */ - private Text getNextRow(final Text row, + private byte [] getNextRow(final byte [] row, final SortedMap map) { - Text result = null; + byte [] result = null; // Synchronize on the map to make the tailMap making 'safe'. synchronized (map) { // Make an HSK with maximum timestamp so we get past most of the current @@ -243,7 +241,7 @@ class Memcache { // Iterate until we fall into the next row; i.e. move off current row for (Map.Entry es: tailMap.entrySet()) { HStoreKey itKey = es.getKey(); - if (itKey.getRow().compareTo(row) <= 0) { + if (Bytes.compareTo(itKey.getRow(), row) <= 0) { continue; } // Note: Not suppressing deletes or expired cells. @@ -262,8 +260,8 @@ class Memcache { * @param deletes Map to accumulate deletes found. * @param results Where to stick row results found. */ - void getFull(HStoreKey key, Set columns, Map deletes, - Map results) { + void getFull(HStoreKey key, Set columns, Map deletes, + Map results) { this.lock.readLock().lock(); try { // The synchronizations here are because internalGet iterates @@ -279,34 +277,31 @@ class Memcache { } private void internalGetFull(SortedMap map, HStoreKey key, - Set columns, Map deletes, Map results) { - + Set columns, Map deletes, + Map results) { if (map.isEmpty() || key == null) { return; } - List victims = new ArrayList(); SortedMap tailMap = map.tailMap(key); long now = System.currentTimeMillis(); for (Map.Entry es: tailMap.entrySet()) { HStoreKey itKey = es.getKey(); - Text itCol = itKey.getColumn(); + byte [] itCol = itKey.getColumn(); if (results.get(itCol) == null && key.matchesWithoutColumn(itKey)) { - byte [] val = tailMap.get(itKey); - if (columns == null || columns.contains(itKey.getColumn())) { + byte [] val = tailMap.get(itKey); if (HLogEdit.isDeleted(val)) { if (!deletes.containsKey(itCol) || deletes.get(itCol).longValue() < itKey.getTimestamp()) { - deletes.put(new Text(itCol), Long.valueOf(itKey.getTimestamp())); + deletes.put(itCol, Long.valueOf(itKey.getTimestamp())); } } else if (!(deletes.containsKey(itCol) && deletes.get(itCol).longValue() >= itKey.getTimestamp())) { // Skip expired cells if (ttl == HConstants.FOREVER || now < itKey.getTimestamp() + ttl) { - results.put(new Text(itCol), - new Cell(val, itKey.getTimestamp())); + results.put(itCol, new Cell(val, itKey.getTimestamp())); } else { victims.add(itKey); if (LOG.isDebugEnabled()) { @@ -315,7 +310,7 @@ class Memcache { } } } - } else if (key.getRow().compareTo(itKey.getRow()) < 0) { + } else if (Bytes.compareTo(key.getRow(), itKey.getRow()) < 0) { break; } } @@ -329,7 +324,7 @@ class Memcache { * @param candidateKeys Map of candidate keys (Accumulation over lots of * lookup over stores and memcaches) */ - void getRowKeyAtOrBefore(final Text row, + void getRowKeyAtOrBefore(final byte [] row, SortedMap candidateKeys) { this.lock.readLock().lock(); try { @@ -345,14 +340,12 @@ class Memcache { } private void internalGetRowKeyAtOrBefore(SortedMap map, - Text key, SortedMap candidateKeys) { - + byte [] key, SortedMap candidateKeys) { HStoreKey strippedKey = null; // we want the earliest possible to start searching from HStoreKey search_key = candidateKeys.isEmpty() ? new HStoreKey(key) : new HStoreKey(candidateKeys.firstKey().getRow()); - Iterator key_iterator = null; HStoreKey found_key = null; ArrayList victims = new ArrayList(); @@ -363,14 +356,15 @@ class Memcache { // if there are items in the tail map, there's either a direct match to // the search key, or a range of values between the first candidate key // and the ultimate search key (or the end of the cache) - if (!tailMap.isEmpty() && tailMap.firstKey().getRow().compareTo(key) <= 0) { + if (!tailMap.isEmpty() && + Bytes.compareTo(tailMap.firstKey().getRow(), key) <= 0) { key_iterator = tailMap.keySet().iterator(); // keep looking at cells as long as they are no greater than the // ultimate search key and there's still records left in the map. do { found_key = key_iterator.next(); - if (found_key.getRow().compareTo(key) <= 0) { + if (Bytes.compareTo(found_key.getRow(), key) <= 0) { strippedKey = stripTimestamp(found_key); if (HLogEdit.isDeleted(tailMap.get(found_key))) { if (candidateKeys.containsKey(strippedKey)) { @@ -393,7 +387,7 @@ class Memcache { } } } - } while (found_key.getRow().compareTo(key) <= 0 + } while (Bytes.compareTo(found_key.getRow(), key) <= 0 && key_iterator.hasNext()); } else { // the tail didn't contain any keys that matched our criteria, or was @@ -412,13 +406,14 @@ class Memcache { HStoreKey[] cells = headMap.keySet().toArray(new HStoreKey[headMap.keySet().size()]); - Text lastRowFound = null; + byte [] lastRowFound = null; for(int i = cells.length - 1; i >= 0; i--) { HStoreKey thisKey = cells[i]; // if the last row we found a candidate key for is different than // the row of the current candidate, we can stop looking. - if (lastRowFound != null && !lastRowFound.equals(thisKey.getRow())) { + if (lastRowFound != null && + !Bytes.equals(lastRowFound, thisKey.getRow())) { break; } @@ -587,10 +582,10 @@ class Memcache { HStoreKey key = es.getKey(); // if there's no column name, then compare rows and timestamps - if (origin.getColumn().toString().equals("")) { + if (origin.getColumn().length == 0) { // if the current and origin row don't match, then we can jump // out of the loop entirely. - if (!key.getRow().equals(origin.getRow())) { + if (!Bytes.equals(key.getRow(), origin.getRow())) { break; } // if the rows match but the timestamp is newer, skip it so we can @@ -644,7 +639,7 @@ class Memcache { * @return a scanner over the keys in the Memcache */ InternalScanner getScanner(long timestamp, - Text targetCols[], Text firstRow) + final byte [][] targetCols, final byte [] firstRow) throws IOException { this.lock.readLock().lock(); try { @@ -660,11 +655,11 @@ class Memcache { ////////////////////////////////////////////////////////////////////////////// private class MemcacheScanner extends HAbstractScanner { - private Text currentRow; - private Set columns = null; + private byte [] currentRow; + private Set columns = null; - MemcacheScanner(final long timestamp, final Text targetCols[], - final Text firstRow) + MemcacheScanner(final long timestamp, final byte [] targetCols[], + final byte [] firstRow) throws IOException { // Call to super will create ColumnMatchers and whether this is a regex // scanner or not. Will also save away timestamp. Also sorts rows. @@ -675,7 +670,7 @@ class Memcache { // columns. this.columns = null; if (!isWildcardScanner()) { - this.columns = new HashSet(); + this.columns = new TreeSet(Bytes.BYTES_COMPARATOR); for (int i = 0; i < targetCols.length; i++) { this.columns.add(targetCols[i]); } @@ -684,15 +679,19 @@ class Memcache { /** {@inheritDoc} */ @Override - public boolean next(HStoreKey key, SortedMap results) + public boolean next(HStoreKey key, SortedMap results) throws IOException { if (this.scannerClosed) { return false; } - Map deletes = new HashMap(); + // This is a treemap rather than a Hashmap because then I can have a + // byte array as key -- because I can independently specify a comparator. + Map deletes = + new TreeMap(Bytes.BYTES_COMPARATOR); // Catch all row results in here. These results are ten filtered to // ensure they match column name regexes, or if none, added to results. - Map rowResults = new HashMap(); + Map rowResults = + new TreeMap(Bytes.BYTES_COMPARATOR); if (results.size() > 0) { results.clear(); } @@ -708,12 +707,12 @@ class Memcache { key.setVersion(this.timestamp); getFull(key, isWildcardScanner() ? null : this.columns, deletes, rowResults); - for (Map.Entry e: deletes.entrySet()) { + for (Map.Entry e: deletes.entrySet()) { rowResults.put(e.getKey(), - new Cell(HLogEdit.deleteBytes.get(), e.getValue())); + new Cell(HLogEdit.deleteBytes.get(), e.getValue().longValue())); } - for (Map.Entry e: rowResults.entrySet()) { - Text column = e.getKey(); + for (Map.Entry e: rowResults.entrySet()) { + byte [] column = e.getKey(); Cell c = e.getValue(); if (isWildcardScanner()) { // Check the results match. We only check columns, not timestamps. @@ -736,4 +735,4 @@ class Memcache { } } } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/regionserver/RegionUnavailableListener.java b/src/java/org/apache/hadoop/hbase/regionserver/RegionUnavailableListener.java index b76afa7d3cb..fac6d1ea2c9 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/RegionUnavailableListener.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/RegionUnavailableListener.java @@ -19,7 +19,6 @@ */ package org.apache.hadoop.hbase.regionserver; -import org.apache.hadoop.io.Text; /** * Used as a callback mechanism so that an HRegion can notify the HRegionServer @@ -33,12 +32,12 @@ public interface RegionUnavailableListener { * outstanding transactions. * @param regionName */ - public void closing(final Text regionName); + public void closing(final byte [] regionName); /** * regionName is closed and no longer available. * Listener should clean up any references to regionName * @param regionName */ - public void closed(final Text regionName); + public void closed(final byte [] regionName); } diff --git a/src/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java b/src/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java index 96bb2622aea..7338c92740d 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java @@ -27,8 +27,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HStoreKey; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.MapFile; -import org.apache.hadoop.io.Text; /** * A scanner that iterates through HStore files @@ -50,7 +50,7 @@ implements ChangedReadersObserver { private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); public StoreFileScanner(final HStore store, final long timestamp, - final Text[] targetCols, final Text firstRow) + final byte [][] targetCols, final byte [] firstRow) throws IOException { super(timestamp, targetCols); this.store = store; @@ -71,7 +71,7 @@ implements ChangedReadersObserver { * @param firstRow * @throws IOException */ - private void openReaders(final Text firstRow) throws IOException { + private void openReaders(final byte [] firstRow) throws IOException { if (this.readers != null) { for (int i = 0; i < this.readers.length; i++) { this.readers[i].close(); @@ -92,7 +92,7 @@ implements ChangedReadersObserver { // Advance the readers to the first pos. for (i = 0; i < readers.length; i++) { keys[i] = new HStoreKey(); - if (firstRow.getLength() != 0) { + if (firstRow != null && firstRow.length != 0) { if (findFirstRow(i, firstRow)) { continue; } @@ -130,7 +130,7 @@ implements ChangedReadersObserver { * @see org.apache.hadoop.hbase.regionserver.InternalScanner#next(org.apache.hadoop.hbase.HStoreKey, java.util.SortedMap) */ @Override - public boolean next(HStoreKey key, SortedMap results) + public boolean next(HStoreKey key, SortedMap results) throws IOException { if (this.scannerClosed) { return false; @@ -145,12 +145,11 @@ implements ChangedReadersObserver { if (viableRow.getRow() != null) { key.setRow(viableRow.getRow()); key.setVersion(viableRow.getTimestamp()); - key.setColumn(new Text("")); for (int i = 0; i < keys.length; i++) { // Fetch the data while ((keys[i] != null) - && (keys[i].getRow().compareTo(viableRow.getRow()) == 0)) { + && (Bytes.compareTo(keys[i].getRow(), viableRow.getRow()) == 0)) { // If we are doing a wild card match or there are multiple matchers // per column, we need to scan all the older versions of this row @@ -164,7 +163,7 @@ implements ChangedReadersObserver { if(columnMatch(i)) { // We only want the first result for any specific family member if(!results.containsKey(keys[i].getColumn())) { - results.put(new Text(keys[i].getColumn()), vals[i]); + results.put(keys[i].getColumn(), vals[i]); insertedItem = true; } } @@ -177,7 +176,7 @@ implements ChangedReadersObserver { // Advance the current scanner beyond the chosen row, to // a valid timestamp, so we're ready next time. while ((keys[i] != null) - && ((keys[i].getRow().compareTo(viableRow.getRow()) <= 0) + && ((Bytes.compareTo(keys[i].getRow(), viableRow.getRow()) <= 0) || (keys[i].getTimestamp() > this.timestamp) || (! columnMatch(i)))) { getNext(i); @@ -192,19 +191,19 @@ implements ChangedReadersObserver { // Data stucture to hold next, viable row (and timestamp). class ViableRow { - private final Text row; + private final byte [] row; private final long ts; - ViableRow(final Text r, final long t) { + ViableRow(final byte [] r, final long t) { this.row = r; this.ts = t; } - public Text getRow() { + byte [] getRow() { return this.row; } - public long getTimestamp() { + long getTimestamp() { return this.ts; } } @@ -215,7 +214,7 @@ implements ChangedReadersObserver { */ private ViableRow getNextViableRow() throws IOException { // Find the next viable row label (and timestamp). - Text viableRow = null; + byte [] viableRow = null; long viableTimestamp = -1; long now = System.currentTimeMillis(); long ttl = store.ttl; @@ -224,11 +223,11 @@ implements ChangedReadersObserver { && (columnMatch(i)) && (keys[i].getTimestamp() <= this.timestamp) && ((viableRow == null) - || (keys[i].getRow().compareTo(viableRow) < 0) - || ((keys[i].getRow().compareTo(viableRow) == 0) + || (Bytes.compareTo(keys[i].getRow(), viableRow) < 0) + || ((Bytes.compareTo(keys[i].getRow(), viableRow) == 0) && (keys[i].getTimestamp() > viableTimestamp)))) { if (ttl == HConstants.FOREVER || now < keys[i].getTimestamp() + ttl) { - viableRow = new Text(keys[i].getRow()); + viableRow = keys[i].getRow(); viableTimestamp = keys[i].getTimestamp(); } else { if (LOG.isDebugEnabled()) { @@ -248,7 +247,7 @@ implements ChangedReadersObserver { * @param firstRow seek to this row * @return true if this is the first row or if the row was not found */ - boolean findFirstRow(int i, Text firstRow) throws IOException { + boolean findFirstRow(int i, final byte [] firstRow) throws IOException { ImmutableBytesWritable ibw = new ImmutableBytesWritable(); HStoreKey firstKey = (HStoreKey)readers[i].getClosest(new HStoreKey(firstRow), ibw); @@ -350,7 +349,8 @@ implements ChangedReadersObserver { // up so future call to next will start here. ViableRow viableRow = getNextViableRow(); openReaders(viableRow.getRow()); - LOG.debug("Replaced Scanner Readers at row " + viableRow.getRow()); + LOG.debug("Replaced Scanner Readers at row " + + Bytes.toString(viableRow.getRow())); } finally { this.lock.writeLock().unlock(); } diff --git a/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java b/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java index 20c8fb85129..5025f8046ee 100644 --- a/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java +++ b/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java @@ -32,13 +32,12 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.io.Text; -import org.mortbay.servlet.MultiPartResponse; import org.znerd.xmlenc.LineBreak; import org.znerd.xmlenc.XMLOutputter; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.util.Bytes; /** * GenericHandler contains some basic common stuff that all the individual @@ -228,13 +227,12 @@ public abstract class GenericHandler { * @throws IOException */ protected void outputColumnsXml(final XMLOutputter outputter, - final Map m) + final Map m) throws IllegalStateException, IllegalArgumentException, IOException { - for (Map.Entry e: m.entrySet()) { + for (Map.Entry e: m.entrySet()) { outputter.startTag(COLUMN); doElement(outputter, "name", - org.apache.hadoop.hbase.util.Base64.encodeBytes( - e.getKey().getBytes())); + org.apache.hadoop.hbase.util.Base64.encodeBytes(e.getKey())); // We don't know String from binary data so we always base64 encode. doElement(outputter, "value", org.apache.hadoop.hbase.util.Base64.encodeBytes(e.getValue().getValue())); @@ -259,6 +257,6 @@ public abstract class GenericHandler { * Get an HTable instance by it's table name. */ protected HTable getTable(final String tableName) throws IOException { - return new HTable(this.conf, new Text(tableName)); + return new HTable(this.conf, Bytes.toBytes(tableName)); } } diff --git a/src/java/org/apache/hadoop/hbase/rest/ScannerHandler.java b/src/java/org/apache/hadoop/hbase/rest/ScannerHandler.java index 87d8b777039..d83631b7247 100644 --- a/src/java/org/apache/hadoop/hbase/rest/ScannerHandler.java +++ b/src/java/org/apache/hadoop/hbase/rest/ScannerHandler.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Scanner; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JenkinsHash; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.RowResult; @@ -178,7 +179,7 @@ public class ScannerHandler extends GenericHandler { // write the row key doElement(outputter, "name", - org.apache.hadoop.hbase.util.Base64.encodeBytes(rowResult.getRow().getBytes())); + org.apache.hadoop.hbase.util.Base64.encodeBytes(rowResult.getRow())); outputColumnsXml(outputter, rowResult); outputter.endTag(); @@ -244,14 +245,14 @@ public class ScannerHandler extends GenericHandler { // get the list of columns we're supposed to interact with String[] raw_columns = request.getParameterValues(COLUMN); - Text [] columns = null; + byte [][] columns = null; if (raw_columns != null) { - columns = new Text [raw_columns.length]; + columns = new byte [raw_columns.length][]; for (int i = 0; i < raw_columns.length; i++) { // I think this decoding is redundant. columns[i] = - new Text(URLDecoder.decode(raw_columns[i], HConstants.UTF8_ENCODING)); + Bytes.toBytes(URLDecoder.decode(raw_columns[i], HConstants.UTF8_ENCODING)); } } else { // TODO: Need to put into the scanner all of the table's column @@ -264,14 +265,14 @@ public class ScannerHandler extends GenericHandler { String raw_ts = request.getParameter(TIMESTAMP); // TODO: Are these decodings redundant? - Text startRow = request.getParameter(START_ROW) == null? + byte [] startRow = request.getParameter(START_ROW) == null? HConstants.EMPTY_START_ROW: - new Text(URLDecoder.decode(request.getParameter(START_ROW), + Bytes.toBytes(URLDecoder.decode(request.getParameter(START_ROW), HConstants.UTF8_ENCODING)); // Empty start row is same value as empty end row. - Text endRow = request.getParameter(END_ROW) == null? + byte [] endRow = request.getParameter(END_ROW) == null? HConstants.EMPTY_START_ROW: - new Text(URLDecoder.decode(request.getParameter(END_ROW), + Bytes.toBytes(URLDecoder.decode(request.getParameter(END_ROW), HConstants.UTF8_ENCODING)); Scanner scanner = (request.getParameter(END_ROW) == null)? diff --git a/src/java/org/apache/hadoop/hbase/rest/TableHandler.java b/src/java/org/apache/hadoop/hbase/rest/TableHandler.java index ac25e70d633..c6bbaf319ef 100644 --- a/src/java/org/apache/hadoop/hbase/rest/TableHandler.java +++ b/src/java/org/apache/hadoop/hbase/rest/TableHandler.java @@ -22,10 +22,11 @@ package org.apache.hadoop.hbase.rest; import java.io.IOException; import java.io.PrintWriter; import java.net.URLDecoder; -import java.util.HashMap; +import java.util.Collection; import java.util.HashSet; import java.util.Map; import java.util.Set; +import java.util.TreeMap; import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; @@ -33,17 +34,16 @@ import javax.servlet.http.HttpServletResponse; import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilderFactory; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.BatchUpdate; - -import org.mortbay.servlet.MultiPartResponse; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.Text; import org.w3c.dom.Document; import org.w3c.dom.Element; import org.w3c.dom.Node; @@ -135,9 +135,9 @@ public class TableHandler extends GenericHandler { // They want full row returned. // Presumption is that this.table has already been focused on target table. - Map result = timestampStr == null ? - table.getRow(new Text(row)) - : table.getRow(new Text(row), Long.parseLong(timestampStr)); + Map result = timestampStr == null ? + table.getRow(Bytes.toBytes(row)) + : table.getRow(Bytes.toBytes(row), Long.parseLong(timestampStr)); if (result == null || result.size() == 0) { doNotFound(response, "Row not found!"); @@ -153,7 +153,7 @@ public class TableHandler extends GenericHandler { } } } else { - Map prefiltered_result = table.getRow(new Text(row)); + Map prefiltered_result = table.getRow(Bytes.toBytes(row)); if (prefiltered_result == null || prefiltered_result.size() == 0) { doNotFound(response, "Row not found!"); @@ -166,16 +166,14 @@ public class TableHandler extends GenericHandler { } // output map that will contain the filtered results - Map m = new HashMap(); + Map m = + new TreeMap(Bytes.BYTES_COMPARATOR); // get an array of all the columns retrieved - Text[] columns_retrieved = - prefiltered_result.keySet().toArray( - new Text[prefiltered_result.keySet().size()]); + Set columns_retrieved = prefiltered_result.keySet(); // copy over those cells with requested column names - for(int i = 0; i < columns_retrieved.length; i++){ - Text current_column = (Text)columns_retrieved[i]; + for(byte [] current_column: columns_retrieved) { if(requested_columns_set.contains(current_column.toString())){ m.put(current_column, prefiltered_result.get(current_column)); } @@ -201,7 +199,7 @@ public class TableHandler extends GenericHandler { * @throws IOException */ private void outputRowXml(final HttpServletResponse response, - final Map result) + final Map result) throws IOException { setResponseHeader(response, result.size() > 0? 200: 204, ContentType.XML.toString()); @@ -349,7 +347,7 @@ public class TableHandler extends GenericHandler { final HttpServletResponse response) throws IOException { // Presumption is that this.table has already been focused on target table. - Text [] startKeys = table.getStartKeys(); + byte [][] startKeys = table.getStartKeys(); // Presumption is that this.table has already been set against target table switch (ContentType.getContentType(request.getHeader(ACCEPT))) { case XML: @@ -410,18 +408,16 @@ public class TableHandler extends GenericHandler { outputter.startTag("table"); doElement(outputter, "name", descriptor.getName().toString()); outputter.startTag("columnfamilies"); - for (Map.Entry e: - descriptor.getFamilies().entrySet()) { + for (HColumnDescriptor e: descriptor.getFamilies()) { outputter.startTag("columnfamily"); - doElement(outputter, "name", e.getKey().toString()); - HColumnDescriptor hcd = e.getValue(); - doElement(outputter, "compression", hcd.getCompression().toString()); + doElement(outputter, "name", Bytes.toString(e.getName())); + doElement(outputter, "compression", e.getCompression().toString()); doElement(outputter, "bloomfilter", - hcd.getBloomFilter() == null? "NONE": hcd.getBloomFilter().toString()); + e.getBloomFilter() == null? "NONE": e.getBloomFilter().toString()); doElement(outputter, "max-versions", - Integer.toString(hcd.getMaxVersions())); + Integer.toString(e.getMaxVersions())); doElement(outputter, "maximum-cell-size", - Integer.toString(hcd.getMaxValueLength())); + Integer.toString(e.getMaxValueLength())); outputter.endTag(); } outputter.endTag(); diff --git a/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java b/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java index 3a42153a1de..2e221134195 100644 --- a/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java +++ b/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.nio.charset.MalformedInputException; import java.util.AbstractMap; import java.util.ArrayList; -import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.TreeMap; @@ -30,13 +29,17 @@ import java.util.Map.Entry; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HStoreKey; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MasterNotRunningException; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Scanner; +import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.thrift.generated.AlreadyExists; import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor; import org.apache.hadoop.hbase.thrift.generated.Hbase; @@ -46,12 +49,8 @@ import org.apache.hadoop.hbase.thrift.generated.Mutation; import org.apache.hadoop.hbase.thrift.generated.NotFound; import org.apache.hadoop.hbase.thrift.generated.RegionDescriptor; import org.apache.hadoop.hbase.thrift.generated.ScanEntry; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.Text; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Scanner; -import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.hbase.io.BatchUpdate; import com.facebook.thrift.TException; import com.facebook.thrift.protocol.TBinaryProtocol; @@ -149,13 +148,13 @@ public class ThriftServer { * @throws IllegalArgument * @throws IOError */ - Text getText(byte[] buf) throws IOError { + byte [] getText(byte[] buf) throws IOError { try { Text.validateUTF8(buf); } catch (MalformedInputException e) { throw new IOError("invalid UTF-8 encoding in row or column name"); } - return new Text(buf); + return buf; } // @@ -183,7 +182,7 @@ public class ThriftServer { LOG.debug("getTableRegions: " + new String(tableName)); try { HTable table = getTable(tableName); - Text[] startKeys = table.getStartKeys(); + byte [][] startKeys = table.getStartKeys(); ArrayList regions = new ArrayList(); for (int i = 0; i < startKeys.length; i++) { RegionDescriptor region = new RegionDescriptor(); @@ -276,12 +275,13 @@ public class ThriftServer { } try { HTable table = getTable(tableName); - Map values = + Map values = table.getRow(getText(row), timestamp); // copy the map from type to - HashMap returnValues = new HashMap(); - for (Entry e : values.entrySet()) { - returnValues.put(e.getKey().getBytes(), e.getValue().getValue()); + TreeMap returnValues = + new TreeMap(Bytes.BYTES_COMPARATOR); + for (Entry e : values.entrySet()) { + returnValues.put(e.getKey(), e.getValue().getValue()); } return returnValues; } catch (IOException e) { @@ -353,11 +353,11 @@ public class ThriftServer { LOG.debug("createTable: table=" + new String(tableName)); } try { - Text tableStr = getText(tableName); + byte [] tableStr = getText(tableName); if (admin.tableExists(tableStr)) { throw new AlreadyExists("table name already in use"); } - HTableDescriptor desc = new HTableDescriptor(tableStr.toString()); + HTableDescriptor desc = new HTableDescriptor(tableStr); for (ColumnDescriptor col : columnFamilies) { HColumnDescriptor colDesc = ThriftUtilities.colDescFromThrift(col); desc.addFamily(colDesc); @@ -378,7 +378,7 @@ public class ThriftServer { LOG.debug("deleteTable: table=" + new String(tableName)); } try { - Text tableStr = getText(tableName); + byte [] tableStr = getText(tableName); if (!admin.tableExists(tableStr)) { throw new NotFound(); } @@ -460,11 +460,11 @@ public class ThriftServer { } ScanEntry retval = new ScanEntry(); - retval.row = results.getRow().getBytes(); - retval.columns = new HashMap(results.size()); + retval.row = results.getRow(); + retval.columns = new TreeMap(Bytes.BYTES_COMPARATOR); - for (Map.Entry e : results.entrySet()) { - retval.columns.put(e.getKey().getBytes(), e.getValue().getValue()); + for (Map.Entry e : results.entrySet()) { + retval.columns.put(e.getKey(), e.getValue().getValue()); } return retval; } @@ -477,7 +477,7 @@ public class ThriftServer { } try { HTable table = getTable(tableName); - Text[] columnsText = new Text[columns.size()]; + byte [][] columnsText = new byte[columns.size()][]; for (int i = 0; i < columns.size(); ++i) { columnsText[i] = getText(columns.get(i)); } @@ -498,7 +498,7 @@ public class ThriftServer { } try { HTable table = getTable(tableName); - Text[] columnsText = new Text[columns.size()]; + byte [][] columnsText = new byte[columns.size()][]; for (int i = 0; i < columns.size(); ++i) { columnsText[i] = getText(columns.get(i)); } @@ -519,7 +519,7 @@ public class ThriftServer { } try { HTable table = getTable(tableName); - Text[] columnsText = new Text[columns.size()]; + byte [][] columnsText = new byte[columns.size()][]; for (int i = 0; i < columns.size(); ++i) { columnsText[i] = getText(columns.get(i)); } @@ -541,7 +541,7 @@ public class ThriftServer { } try { HTable table = getTable(tableName); - Text[] columnsText = new Text[columns.size()]; + byte [][] columnsText = new byte[columns.size()][]; for (int i = 0; i < columns.size(); ++i) { columnsText[i] = getText(columns.get(i)); } @@ -559,13 +559,14 @@ public class ThriftServer { LOG.debug("getColumnDescriptors: table=" + new String(tableName)); } try { - HashMap columns = new HashMap(); + TreeMap columns = + new TreeMap(Bytes.BYTES_COMPARATOR); HTable table = getTable(tableName); HTableDescriptor desc = table.getMetadata(); - for (Entry e : desc.families().entrySet()) { - ColumnDescriptor col = ThriftUtilities.colDescFromHbase(e.getValue()); + for (HColumnDescriptor e : desc.getFamilies()) { + ColumnDescriptor col = ThriftUtilities.colDescFromHbase(e); columns.put(col.name, col); } return columns; diff --git a/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java b/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java index 029c1ed5a93..4d55484125e 100644 --- a/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java +++ b/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java @@ -23,7 +23,6 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType; import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor; import org.apache.hadoop.hbase.thrift.generated.IllegalArgument; -import org.apache.hadoop.io.Text; public class ThriftUtilities { @@ -57,7 +56,7 @@ public class ThriftUtilities { if (in.name == null || in.name.length <= 0) { throw new IllegalArgument("column name is empty"); } - HColumnDescriptor col = new HColumnDescriptor(new Text(in.name), + HColumnDescriptor col = new HColumnDescriptor(in.name, in.maxVersions, comp, in.inMemory, in.blockCacheEnabled, in.maxValueLength, in.timeToLive, bloom); return col; @@ -73,7 +72,7 @@ public class ThriftUtilities { */ static public ColumnDescriptor colDescFromHbase(HColumnDescriptor in) { ColumnDescriptor col = new ColumnDescriptor(); - col.name = in.getName().getBytes(); + col.name = in.getName(); col.maxVersions = in.getMaxVersions(); col.compression = in.getCompression().toString(); col.inMemory = in.isInMemory(); @@ -88,4 +87,4 @@ public class ThriftUtilities { return col; } -} +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/util/Bytes.java b/src/java/org/apache/hadoop/hbase/util/Bytes.java new file mode 100644 index 00000000000..209d4ce9614 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/Bytes.java @@ -0,0 +1,269 @@ +package org.apache.hadoop.hbase.util; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.nio.ByteBuffer; +import java.util.Comparator; + +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.WritableComparator; + +public class Bytes { + /** + * Size of long in bytes + */ + public static final int SIZEOF_LONG = Long.SIZE/Byte.SIZE; + + /** + * Size of int in bytes + */ + public static final int SIZEOF_INT = Integer.SIZE/Byte.SIZE; + + /** + * Pass this to TreeMaps where byte [] are keys. + */ + public static Comparator BYTES_COMPARATOR = + new Comparator() { + public int compare(byte [] left, byte [] right) { + return compareTo(left, right); + } + }; + + /** + * @param in Input to read from. + * @return byte array read off in + * @throws IOException + */ + public static byte [] readByteArray(final DataInput in) + throws IOException { + byte [] result = new byte[in.readInt()]; + in.readFully(result, 0, result.length); + return result; + } + + /** + * @param out + * @param b + * @throws IOException + */ + public static void writeByteArray(final DataOutput out, final byte [] b) + throws IOException { + out.writeInt(b.length); + out.write(b, 0, b.length); + } + + /** + * @param b Presumed UTF-8 encoded byte array. + * @return String made from b + */ + public static String toString(final byte [] b) { + String result = null; + try { + result = new String(b, HConstants.UTF8_ENCODING); + } catch (UnsupportedEncodingException e) { + e.printStackTrace(); + } + return result; + } + + + /** + * Converts a string to a UTF-8 byte array. + * @param s + * @return the byte array + */ + public static byte[] toBytes(String s) { + if (s == null) { + throw new IllegalArgumentException("string cannot be null"); + } + byte [] result = null; + try { + result = s.getBytes(HConstants.UTF8_ENCODING); + } catch (UnsupportedEncodingException e) { + e.printStackTrace(); + } + return result; + } + + /** + * Convert a long value to a byte array + * @param val + * @return the byte array + */ + public static byte[] toBytes(final long val) { + ByteBuffer bb = ByteBuffer.allocate(SIZEOF_LONG); + bb.putLong(val); + return bb.array(); + } + + /** + * Converts a byte array to a long value + * @param bytes + * @return the long value + */ + public static long toLong(byte[] bytes) { + if (bytes == null || bytes.length == 0) { + return -1L; + } + return ByteBuffer.wrap(bytes).getLong(); + } + + /** + * Convert an int value to a byte array + * @param val + * @return the byte array + */ + public static byte[] toBytes(final int val) { + ByteBuffer bb = ByteBuffer.allocate(SIZEOF_INT); + bb.putInt(val); + return bb.array(); + } + + /** + * Converts a byte array to a long value + * @param bytes + * @return the long value + */ + public static long toInt(byte[] bytes) { + if (bytes == null || bytes.length == 0) { + return -1L; + } + return ByteBuffer.wrap(bytes).getInt(); + } + + /** + * @param left + * @param right + * @return 0 if equal, < 0 if left is less than right, etc. + */ + public static int compareTo(final byte [] left, final byte [] right) { + return compareTo(left, 0, left.length, right, 0, right.length); + } + + /** + * @param left + * @param right + * @param leftOffset Where to start comparing in the left buffer + * @param rightOffset Where to start comparing in the right buffer + * @param leftLength How much to compare from the left buffer + * @param rightLength How much to compare from the right buffer + * @return 0 if equal, < 0 if left is less than right, etc. + */ + public static int compareTo(final byte [] left, final int leftOffset, + final int leftLength, final byte [] right, final int rightOffset, + final int rightLength) { + return WritableComparator.compareBytes(left,leftOffset, leftLength, + right, rightOffset, rightLength); + } + + /** + * @param left + * @param right + * @return True if equal + */ + public static boolean equals(final byte [] left, final byte [] right) { + return left == null && right == null? true: + left == null && right != null? false: + left != null && right == null? false: + left.length != right.length? false: + compareTo(left, right) == 0; + } + + /** + * @param b + * @return Runs {@link WritableComparator#hashBytes(byte[], int)} on the + * passed in array. This method is what {@link Text} and + * {@link ImmutableBytesWritable} use calculating hash code. + */ + public static int hashCode(final byte [] b) { + return hashCode(b, b.length); + } + + /** + * @param b + * @return Runs {@link WritableComparator#hashBytes(byte[], int)} on the + * passed in array. This method is what {@link Text} and + * {@link ImmutableBytesWritable} use calculating hash code. + */ + public static int hashCode(final byte [] b, final int length) { + return WritableComparator.hashBytes(b, length); + } + + /** + * @param b + * @return A hash of b as an Integer that can be used as key in + * Maps. + */ + public static Integer mapKey(final byte [] b) { + return Integer.valueOf(hashCode(b)); + } + + /** + * @param b + * @return A hash of b as an Integer that can be used as key in + * Maps. + */ + public static Integer mapKey(final byte [] b, final int length) { + return Integer.valueOf(hashCode(b, length)); + } + + /** + * @param a + * @param b + * @return New array that has a in lower half and b in upper half. + */ + public static byte [] add(final byte [] a, final byte [] b) { + return add(a, b, HConstants.EMPTY_BYTE_ARRAY); + } + + /** + * @param a + * @param b + * @param c + * @return New array made from a, b and c + */ + public static byte [] add(final byte [] a, final byte [] b, final byte [] c) { + byte [] result = new byte[a.length + b.length + c.length]; + System.arraycopy(a, 0, result, 0, a.length); + System.arraycopy(b, 0, result, a.length, b.length); + System.arraycopy(c, 0, result, a.length + b.length, c.length); + return result; + } + + + /** + * @param t + * @return Array of byte arrays made from passed array of Text + */ + public static byte [][] toByteArrays(final Text [] t) { + byte [][] result = new byte[t.length][]; + for (int i = 0; i < t.length; i++) { + result[i] = t[i].getBytes(); + } + return result; + } + + /** + * @param column + * @return A byte array of a byte array where first and only entry is + * column + */ + public static byte [][] toByteArrays(final String column) { + return toByteArrays(toBytes(column)); + } + + /** + * @param column + * @return A byte array of a byte array where first and only entry is + * column + */ + public static byte [][] toByteArrays(final byte [] column) { + byte [][] result = new byte[1][]; + result[0] = column; + return result; + } +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/util/JenkinsHash.java b/src/java/org/apache/hadoop/hbase/util/JenkinsHash.java index cefb7309444..28f39f73e74 100644 --- a/src/java/org/apache/hadoop/hbase/util/JenkinsHash.java +++ b/src/java/org/apache/hadoop/hbase/util/JenkinsHash.java @@ -24,13 +24,19 @@ import java.io.FileInputStream; import java.io.IOException; /** - * lookup3.c, by Bob Jenkins, May 2006, Public Domain. - * lookup3.c + * Produces 32-bit hash for hash table lookup. + * + *

lookup3.c, by Bob Jenkins, May 2006, Public Domain.
  *
  * You can use this free for any purpose.  It's in the public domain.
  * It has no warranty.
+ * 
* - * Produces 32-bit hash for hash table lookup. + * @see lookup3.c + * @see Hash Functions (and how this + * function compares to others such as CRC, MD?, etc + * @see Has update on the + * Dr. Dobbs Article */ public class JenkinsHash { private static long INT_MASK = 0x00000000ffffffffL; @@ -41,6 +47,16 @@ public class JenkinsHash { Long.valueOf(val & INT_MASK).intValue(), pos)).longValue() & INT_MASK; } + /** + * Alternate form for hashing an entire byte array + * + * @param bytes + * @return hash value + */ + public static int hash(byte[] bytes) { + return hash(bytes, bytes.length, -1); + } + /** * Alternate form for hashing an entire byte array * @@ -62,18 +78,19 @@ public class JenkinsHash { * return value. Two keys differing by one or two bits will have totally * different hash values. * - * The best hash table sizes are powers of 2. There is no need to do mod a - * prime (mod is sooo slow!). If you need less than 32 bits, use a bitmask. - * For example, if you need only 10 bits, do h = (h & hashmask(10)); + *

The best hash table sizes are powers of 2. There is no need to do mod + * a prime (mod is sooo slow!). If you need less than 32 bits, use a bitmask. + * For example, if you need only 10 bits, do + * h = (h & hashmask(10)); * In which case, the hash table should have hashsize(10) elements. * - * If you are hashing n strings byte[][] k, do it like this: + *

If you are hashing n strings byte[][] k, do it like this: * for (int i = 0, h = 0; i < n; ++i) h = hash( k[i], h); * - * By Bob Jenkins, 2006. bob_jenkins@burtleburtle.net. You may use this + *

By Bob Jenkins, 2006. bob_jenkins@burtleburtle.net. You may use this * code any way you wish, private, educational, or commercial. It's free. * - * Use for hash table lookup, or anything where one collision in 2^^32 is + *

Use for hash table lookup, or anything where one collision in 2^^32 is * acceptable. Do NOT use for cryptographic purposes. */ public static int hash(byte[] key, int nbytes, int initval) { diff --git a/src/java/org/apache/hadoop/hbase/util/Merge.java b/src/java/org/apache/hadoop/hbase/util/Merge.java index eaf36cb4157..e03ef7c28df 100644 --- a/src/java/org/apache/hadoop/hbase/util/Merge.java +++ b/src/java/org/apache/hadoop/hbase/util/Merge.java @@ -27,7 +27,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableComparator; import org.apache.hadoop.util.GenericOptionsParser; import org.apache.hadoop.util.Tool; @@ -50,9 +49,9 @@ public class Merge extends Configured implements Tool { private final HBaseConfiguration conf; private Path rootdir; private volatile MetaUtils utils; - private Text tableName; // Name of table - private volatile Text region1; // Name of region 1 - private volatile Text region2; // Name of region 2 + private byte [] tableName; // Name of table + private volatile byte [] region1; // Name of region 1 + private volatile byte [] region2; // Name of region 2 private volatile boolean isMetaTable; private volatile HRegionInfo mergeInfo; @@ -109,7 +108,7 @@ public class Merge extends Configured implements Tool { return 0; } catch (Exception e) { LOG.fatal("Merge failed", e); - utils.scanMetaRegion(HRegionInfo.firstMetaRegionInfo, + utils.scanMetaRegion(HRegionInfo.FIRST_META_REGIONINFO, new MetaUtils.ScannerListener() { public boolean processRow(HRegionInfo info) { System.err.println(info.toString()); @@ -154,12 +153,12 @@ public class Merge extends Configured implements Tool { private static class MetaScannerListener implements MetaUtils.ScannerListener { - private final Text region1; - private final Text region2; + private final byte [] region1; + private final byte [] region2; private HRegionInfo meta1 = null; private HRegionInfo meta2 = null; - MetaScannerListener(Text region1, Text region2) { + MetaScannerListener(final byte [] region1, final byte [] region2) { this.region1 = region1; this.region2 = region2; } @@ -214,7 +213,7 @@ public class Merge extends Configured implements Tool { } HRegion metaRegion2 = null; - if (meta1.getRegionName().equals(meta2.getRegionName())) { + if (Bytes.equals(meta1.getRegionName(), meta2.getRegionName())) { metaRegion2 = metaRegion1; } else { metaRegion2 = utils.getMetaRegion(meta2); @@ -236,9 +235,9 @@ public class Merge extends Configured implements Tool { merged.getRegionName()); } HRegion mergeMeta = null; - if (mergedInfo.getRegionName().equals(meta1.getRegionName())) { + if (Bytes.equals(mergedInfo.getRegionName(), meta1.getRegionName())) { mergeMeta = metaRegion1; - } else if (mergedInfo.getRegionName().equals(meta2.getRegionName())) { + } else if (Bytes.equals(mergedInfo.getRegionName(), meta2.getRegionName())) { mergeMeta = metaRegion2; } else { mergeMeta = utils.getMetaRegion(mergedInfo); @@ -330,32 +329,30 @@ public class Merge extends Configured implements Tool { usage(); return -1; } - tableName = new Text(remainingArgs[0]); - isMetaTable = tableName.compareTo(HConstants.META_TABLE_NAME) == 0; + tableName = Bytes.toBytes(remainingArgs[0]); + isMetaTable = Bytes.compareTo(tableName, HConstants.META_TABLE_NAME) == 0; - region1 = new Text(remainingArgs[1]); - region2 = new Text(remainingArgs[2]); + region1 = Bytes.toBytes(remainingArgs[1]); + region2 = Bytes.toBytes(remainingArgs[2]); int status = 0; - // Why we duplicate code here? St.Ack - if (WritableComparator.compareBytes( - tableName.getBytes(), 0, tableName.getLength(), - region1.getBytes(), 0, tableName.getLength()) != 0) { - LOG.error("Region " + region1 + " does not belong to table " + tableName); + if (notInTable(tableName, region1) || notInTable(tableName, region2)) { status = -1; - } - if (WritableComparator.compareBytes( - tableName.getBytes(), 0, tableName.getLength(), - region2.getBytes(), 0, tableName.getLength()) != 0) { - LOG.error("Region " + region2 + " does not belong to table " + tableName); - status = -1; - } - if (region1.equals(region2)) { + } else if (Bytes.equals(region1, region2)) { LOG.error("Can't merge a region with itself"); status = -1; } return status; } + private boolean notInTable(final byte [] tn, final byte [] rn) { + if (WritableComparator.compareBytes(tn, 0, tn.length, rn, 0, tn.length) != 0) { + LOG.error("Region " + Bytes.toString(rn) + " does not belong to table " + + Bytes.toString(tn)); + return true; + } + return false; + } + private void usage() { System.err.println( "Usage: bin/hbase merge \n"); diff --git a/src/java/org/apache/hadoop/hbase/util/MetaUtils.java b/src/java/org/apache/hadoop/hbase/util/MetaUtils.java index 9e1e6fa235c..6e5d7e51a4c 100644 --- a/src/java/org/apache/hadoop/hbase/util/MetaUtils.java +++ b/src/java/org/apache/hadoop/hbase/util/MetaUtils.java @@ -22,9 +22,10 @@ package org.apache.hadoop.hbase.util; import java.io.FileNotFoundException; import java.io.IOException; +import java.util.Collections; +import java.util.Map; import java.util.SortedMap; import java.util.TreeMap; -import java.util.concurrent.ConcurrentHashMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -40,7 +41,6 @@ import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.regionserver.HLog; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.io.Text; /** * Contains utility methods for manipulating HBase meta tables @@ -54,7 +54,8 @@ public class MetaUtils { private Path rootdir; private HLog log; private HRegion rootRegion; - private ConcurrentHashMap metaRegions; + private Map metaRegions = Collections.synchronizedSortedMap( + new TreeMap(Bytes.BYTES_COMPARATOR)); /** Default constructor */ public MetaUtils() { @@ -67,7 +68,6 @@ public class MetaUtils { conf.setInt("hbase.client.retries.number", 1); this.initialized = false; this.rootRegion = null; - this.metaRegions = new ConcurrentHashMap(); } /** @@ -220,7 +220,8 @@ public class MetaUtils { try { HStoreKey key = new HStoreKey(); - SortedMap results = new TreeMap(); + SortedMap results = + new TreeMap(Bytes.BYTES_COMPARATOR); while (rootScanner.next(key, results)) { HRegionInfo info = Writables.getHRegionInfoOrNull( results.get(HConstants.COL_REGIONINFO)); @@ -234,7 +235,6 @@ public class MetaUtils { } results.clear(); } - } finally { rootScanner.close(); } @@ -252,28 +252,38 @@ public class MetaUtils { * @throws IOException */ public void scanMetaRegion(HRegionInfo metaRegionInfo, - ScannerListener listener) throws IOException { + ScannerListener listener) + throws IOException { if (!initialized) { throw new IllegalStateException("Must call initialize method first."); } - // Open meta region so we can scan it - HRegion metaRegion = openMetaRegion(metaRegionInfo); - - InternalScanner metaScanner = metaRegion.getScanner( - HConstants.COL_REGIONINFO_ARRAY, HConstants.EMPTY_START_ROW, - HConstants.LATEST_TIMESTAMP, null); - + scanMetaRegion(metaRegion, listener); + } + + /** + * Scan the passed in metaregion m invoking the passed + * listener per row found. + * @param m + * @param listener + * @throws IOException + */ + public void scanMetaRegion(final HRegion m, + final ScannerListener listener) + throws IOException { + InternalScanner metaScanner = m.getScanner(HConstants.COL_REGIONINFO_ARRAY, + HConstants.EMPTY_START_ROW, HConstants.LATEST_TIMESTAMP, null); try { HStoreKey key = new HStoreKey(); - SortedMap results = new TreeMap(); + SortedMap results = + new TreeMap(Bytes.BYTES_COMPARATOR); while (metaScanner.next(key, results)) { - HRegionInfo info = Writables.getHRegionInfoOrNull( - results.get(HConstants.COL_REGIONINFO)); + HRegionInfo info = + Writables.getHRegionInfoOrNull(results.get(HConstants.COL_REGIONINFO)); if (info == null) { - LOG.warn("region info is null for row " + key.getRow() + - " in table " + HConstants.META_TABLE_NAME); + LOG.warn("regioninfo null for row " + key.getRow() + " in table " + + Bytes.toString(m.getTableDesc().getName())); continue; } if (!listener.processRow(info)) { @@ -281,14 +291,13 @@ public class MetaUtils { } results.clear(); } - } finally { metaScanner.close(); } } private void openRootRegion() throws IOException { - this.rootRegion = HRegion.openHRegion(HRegionInfo.rootRegionInfo, + this.rootRegion = HRegion.openHRegion(HRegionInfo.ROOT_REGIONINFO, this.rootdir, this.log, this.conf); this.rootRegion.compactStores(); } @@ -314,7 +323,7 @@ public class MetaUtils { * @throws IOException */ public static void changeOnlineStatus (final HBaseConfiguration c, - final Text row, final boolean onlineOffline) + final byte [] row, final boolean onlineOffline) throws IOException { HTable t = new HTable(c, HConstants.META_TABLE_NAME); Cell cell = t.get(row, HConstants.COL_REGIONINFO); diff --git a/src/java/org/apache/hadoop/hbase/util/Migrate.java b/src/java/org/apache/hadoop/hbase/util/Migrate.java index 52320efd436..aa87eaef12a 100644 --- a/src/java/org/apache/hadoop/hbase/util/Migrate.java +++ b/src/java/org/apache/hadoop/hbase/util/Migrate.java @@ -44,8 +44,6 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Text; - import org.apache.hadoop.util.GenericOptionsParser; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; @@ -231,7 +229,7 @@ public class Migrate extends Configured implements Tool { // find root region String rootRegion = OLD_PREFIX + - HRegionInfo.rootRegionInfo.getEncodedName(); + HRegionInfo.ROOT_REGIONINFO.getEncodedName(); if (!fs.exists(new Path(rootdir, rootRegion))) { throw new IOException("Cannot find root region " + rootRegion); @@ -264,7 +262,7 @@ public class Migrate extends Configured implements Tool { private void checkNewRootRegionDirExists() throws IOException { Path rootRegionDir = - HRegion.getRegionDir(rootdir, HRegionInfo.rootRegionInfo); + HRegion.getRegionDir(rootdir, HRegionInfo.ROOT_REGIONINFO); newRootRegion = fs.exists(rootRegionDir); migrationNeeded = !newRootRegion; } @@ -340,8 +338,8 @@ public class Migrate extends Configured implements Tool { } } - void migrateRegionDir(Text tableName, String oldPath)throws IOException { - + void migrateRegionDir(final byte [] tableName, String oldPath) + throws IOException { // Create directory where table will live Path tableDir = new Path(rootdir, tableName.toString()); diff --git a/src/java/org/apache/hadoop/hbase/util/SoftSortedMap.java b/src/java/org/apache/hadoop/hbase/util/SoftSortedMap.java index 07fe91f0f65..e6b777e107e 100644 --- a/src/java/org/apache/hadoop/hbase/util/SoftSortedMap.java +++ b/src/java/org/apache/hadoop/hbase/util/SoftSortedMap.java @@ -38,15 +38,22 @@ import org.apache.commons.logging.LogFactory; * play well with the GC when in a low-memory situation. */ public class SoftSortedMap implements SortedMap { - protected static final Log LOG = LogFactory.getLog(SoftSortedMap.class); - - protected SortedMap> internalMap = - new TreeMap>(); - - protected ReferenceQueue referenceQueue = new ReferenceQueue(); + private static final Log LOG = LogFactory.getLog(SoftSortedMap.class); + private final SortedMap> internalMap; + private ReferenceQueue referenceQueue = new ReferenceQueue(); /** Constructor */ - public SoftSortedMap() {} + public SoftSortedMap() { + this(new TreeMap>()); + } + + /** + * Constructor + * @param c + */ + public SoftSortedMap(final Comparator c) { + this(new TreeMap>(c)); + } /** For headMap and tailMap support */ private SoftSortedMap(SortedMap> original) { @@ -61,24 +68,22 @@ public class SoftSortedMap implements SortedMap { return oldValue == null ? null : oldValue.get(); } - public void putAll(Map map) { + @SuppressWarnings("unchecked") + public void putAll(@SuppressWarnings("unused") Map map) { throw new RuntimeException("Not implemented"); } public V get(Object key) { checkReferences(); SoftValue value = internalMap.get(key); - if (value == null) { return null; - } else { - if (value.get() == null) { - internalMap.remove(key); - return null; - } else { - return value.get(); - } } + if (value.get() == null) { + internalMap.remove(key); + return null; + } + return value.get(); } public V remove(Object key) { @@ -92,7 +97,7 @@ public class SoftSortedMap implements SortedMap { return internalMap.containsKey(key); } - public boolean containsValue(Object value) { + public boolean containsValue(@SuppressWarnings("unused") Object value) { /* checkReferences(); return internalMap.containsValue(value);*/ throw new UnsupportedOperationException("Don't support containsValue!"); @@ -141,6 +146,7 @@ public class SoftSortedMap implements SortedMap { return internalMap.keySet(); } + @SuppressWarnings("unchecked") public Comparator comparator() { return internalMap.comparator(); } @@ -169,11 +175,13 @@ public class SoftSortedMap implements SortedMap { */ private void checkReferences() { SoftValue sv; - while((sv = (SoftValue)referenceQueue.poll()) != null) { + Object obj; + while((obj = referenceQueue.poll()) != null) { if (LOG.isDebugEnabled()) { - LOG.debug("Reference for key " + sv.key.toString() + " has been cleared."); + LOG.debug("Reference for key " + ((SoftValue)obj).key.toString() + + " has been cleared."); } - internalMap.remove(sv.key); + internalMap.remove(((SoftValue)obj).key); } } diff --git a/src/java/org/apache/hadoop/hbase/util/Writables.java b/src/java/org/apache/hadoop/hbase/util/Writables.java index 09b6fe0d333..56752e3352b 100644 --- a/src/java/org/apache/hadoop/hbase/util/Writables.java +++ b/src/java/org/apache/hadoop/hbase/util/Writables.java @@ -22,14 +22,11 @@ import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; -import java.io.UnsupportedEncodingException; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.io.DataInputBuffer; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Writable; import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.io.DataInputBuffer; +import org.apache.hadoop.io.Writable; /** * Utility class with methods for manipulating Writable objects @@ -121,9 +118,8 @@ public class Writables { public static HRegionInfo getHRegionInfo(final Cell cell) throws IOException { if (cell == null) { return null; - } else { - return getHRegionInfo(cell.getValue()); } + return getHRegionInfo(cell.getValue()); } /** @@ -135,10 +131,18 @@ public class Writables { */ public static Writable copyWritable(final Writable src, final Writable tgt) throws IOException { - if (src == null || tgt == null) { - throw new IllegalArgumentException("Writables cannot be null"); - } - byte [] bytes = getBytes(src); + return copyWritable(getBytes(src), tgt); + } + + /** + * Copy one Writable to another. Copies bytes using data streams. + * @param bytes Source Writable + * @param tgt Target Writable + * @return The target Writable. + * @throws IOException + */ + public static Writable copyWritable(final byte [] bytes, final Writable tgt) + throws IOException { DataInputStream dis = new DataInputStream(new ByteArrayInputStream(bytes)); try { tgt.readFields(dis); @@ -147,73 +151,26 @@ public class Writables { } return tgt; } - + /** - * Convert a long value to a byte array - * @param val - * @return the byte array - * @throws IOException + * @param c + * @return Cell value as a UTF-8 String */ - public static byte[] longToBytes(long val) throws IOException { - return getBytes(new LongWritable(val)); - } - - /** - * Converts a byte array to a long value - * @param bytes - * @return the long value - * @throws IOException - */ - public static long bytesToLong(byte[] bytes) throws IOException { - if (bytes == null || bytes.length == 0) { - return -1L; - } - return ((LongWritable) getWritable(bytes, new LongWritable())).get(); - } - - /** - * Converts a string to a byte array in a consistent manner. - * @param s - * @return the byte array - * @throws UnsupportedEncodingException - */ - public static byte[] stringToBytes(String s) - throws UnsupportedEncodingException { - if (s == null) { - throw new IllegalArgumentException("string cannot be null"); - } - return s.getBytes(HConstants.UTF8_ENCODING); - } - - /** - * Converts a byte array to a string in a consistent manner. - * @param bytes - * @return the string - * @throws UnsupportedEncodingException - */ - public static String bytesToString(byte[] bytes) - throws UnsupportedEncodingException { - if (bytes == null || bytes.length == 0) { - return ""; - } - return new String(bytes, HConstants.UTF8_ENCODING); - } - - public static String cellToString(Cell c) - throws UnsupportedEncodingException { + public static String cellToString(Cell c) { if (c == null) { return ""; - } else { - return bytesToString(c.getValue()); } + return Bytes.toString(c.getValue()); } - public static long cellToLong(Cell c) - throws IOException { + /** + * @param c + * @return Cell as a long. + */ + public static long cellToLong(Cell c) { if (c == null) { return 0; - } else { - return bytesToLong(c.getValue()); } + return Bytes.toLong(c.getValue()); } -} +} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java b/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java index d9f7abb5486..44c734001f1 100644 --- a/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java +++ b/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java @@ -23,19 +23,17 @@ import java.io.IOException; import java.io.UnsupportedEncodingException; import java.util.Random; -import org.apache.hadoop.dfs.MiniDFSCluster; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.io.Text; -import org.apache.log4j.Logger; - +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.log4j.Logger; /** Abstract base class for merge tests */ public abstract class AbstractMergeTestBase extends HBaseClusterTestCase { static final Logger LOG = Logger.getLogger(AbstractMergeTestBase.class.getName()); - protected static final Text COLUMN_NAME = new Text("contents:"); + protected static final byte [] COLUMN_NAME = Bytes.toBytes("contents:"); protected final Random rand = new Random(); protected HTableDescriptor desc; protected ImmutableBytesWritable value; @@ -65,8 +63,8 @@ public abstract class AbstractMergeTestBase extends HBaseClusterTestCase { } catch (UnsupportedEncodingException e) { fail(); } - desc = new HTableDescriptor("test"); - desc.addFamily(new HColumnDescriptor(COLUMN_NAME.toString())); + desc = new HTableDescriptor(Bytes.toBytes("test")); + desc.addFamily(new HColumnDescriptor(COLUMN_NAME)); } @Override @@ -90,8 +88,8 @@ public abstract class AbstractMergeTestBase extends HBaseClusterTestCase { // To ensure that the first region is larger than 64MB we need to write at // least 65536 rows. We will make certain by writing 70000 - Text row_70001 = new Text("row_70001"); - Text row_80001 = new Text("row_80001"); + byte [] row_70001 = Bytes.toBytes("row_70001"); + byte [] row_80001 = Bytes.toBytes("row_80001"); HRegion[] regions = { createAregion(null, row_70001, 1, 70000), @@ -102,9 +100,9 @@ public abstract class AbstractMergeTestBase extends HBaseClusterTestCase { // Now create the root and meta regions and insert the data regions // created above into the meta - HRegion root = HRegion.createHRegion(HRegionInfo.rootRegionInfo, + HRegion root = HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO, testDir, this.conf); - HRegion meta = HRegion.createHRegion(HRegionInfo.firstMetaRegionInfo, + HRegion meta = HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO, testDir, this.conf); HRegion.addRegionToMETA(root, meta); @@ -118,16 +116,17 @@ public abstract class AbstractMergeTestBase extends HBaseClusterTestCase { meta.getLog().closeAndDelete(); } - private HRegion createAregion(Text startKey, Text endKey, int firstRow, + private HRegion createAregion(byte [] startKey, byte [] endKey, int firstRow, int nrows) throws IOException { HRegion region = createNewHRegion(desc, startKey, endKey); - System.out.println("created region " + region.getRegionName()); + System.out.println("created region " + + Bytes.toString(region.getRegionName())); HRegionIncommon r = new HRegionIncommon(region); for(int i = firstRow; i < firstRow + nrows; i++) { - BatchUpdate batchUpdate = new BatchUpdate(new Text("row_" + BatchUpdate batchUpdate = new BatchUpdate(Bytes.toBytes("row_" + String.format("%1$05d", i))); batchUpdate.put(COLUMN_NAME, value.get()); diff --git a/src/test/org/apache/hadoop/hbase/DisabledTestScanner2.java b/src/test/org/apache/hadoop/hbase/DisabledTestScanner2.java index cf24b6d1b27..a4b055fe1c1 100644 --- a/src/test/org/apache/hadoop/hbase/DisabledTestScanner2.java +++ b/src/test/org/apache/hadoop/hbase/DisabledTestScanner2.java @@ -26,39 +26,35 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.SortedMap; import java.util.TreeMap; import java.util.regex.Pattern; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Scanner; import org.apache.hadoop.hbase.filter.RegExpRowFilter; import org.apache.hadoop.hbase.filter.RowFilterInterface; import org.apache.hadoop.hbase.filter.RowFilterSet; import org.apache.hadoop.hbase.filter.StopRowFilter; import org.apache.hadoop.hbase.filter.WhileMatchRowFilter; -import org.apache.hadoop.hbase.io.HbaseMapWritable; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.Scanner; - -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.ipc.HRegionInterface; -import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Writables; /** * Additional scanner tests. - * {@link TestScanner} does a custom setup/takedown not conducive + * {@link org.apache.hadoop.hbase.regionserver.TestScanner} does a custom + * setup/takedown not conducive * to addition of extra scanning tests. * *

Temporarily disabled until hudson stabilizes again. - * @see TestScanner + * @see org.apache.hadoop.hbase.regionserver.TestScanner */ public class DisabledTestScanner2 extends HBaseClusterTestCase { final Log LOG = LogFactory.getLog(this.getClass().getName()); @@ -90,9 +86,9 @@ public class DisabledTestScanner2 extends HBaseClusterTestCase { */ public void testScanningMultipleFamiliesOfDifferentVintage() throws MasterNotRunningException, IOException { - Text tableName = new Text(getName()); - final Text [] families = createTable(new HBaseAdmin(this.conf), tableName); - HTable table = new HTable(this.conf, tableName); + final byte [][] families = createTable(new HBaseAdmin(this.conf), + getName()); + HTable table = new HTable(this.conf, getName()); Scanner scanner = null; try { long time = System.currentTimeMillis(); @@ -101,20 +97,20 @@ public class DisabledTestScanner2 extends HBaseClusterTestCase { final byte [] lastKey = new byte [] {'a', 'a', (byte)('b' + i)}; Incommon inc = new HTableIncommon(table); addContent(inc, families[i].toString(), - START_KEY_BYTES, new Text(lastKey), time + (1000 * i)); + START_KEY_BYTES, lastKey, time + (1000 * i)); // Add in to the first store a record that is in excess of the stop // row specified below setting up the scanner filter. Add 'bbb'. // Use a stop filter of 'aad'. The store scanner going to 'bbb' was // flipping the switch in StopRowFilter stopping us returning all // of the rest of the other store content. if (i == 0) { - BatchUpdate batchUpdate = new BatchUpdate(new Text("bbb")); + BatchUpdate batchUpdate = new BatchUpdate(Bytes.toBytes("bbb")); batchUpdate.put(families[0], "bbb".getBytes()); inc.commit(batchUpdate); } } RowFilterInterface f = - new WhileMatchRowFilter(new StopRowFilter(new Text("aad"))); + new WhileMatchRowFilter(new StopRowFilter(Bytes.toBytes("aad"))); scanner = table.getScanner(families, HConstants.EMPTY_START_ROW, HConstants.LATEST_TIMESTAMP, f); int count = 0; @@ -132,14 +128,14 @@ public class DisabledTestScanner2 extends HBaseClusterTestCase { * @throws Exception */ public void testStopRow() throws Exception { - Text tableName = new Text(getName()); - createTable(new HBaseAdmin(this.conf), tableName); - HTable table = new HTable(this.conf, tableName); + createTable(new HBaseAdmin(this.conf), getName()); + HTable table = new HTable(this.conf, getName()); final String lastKey = "aac"; addContent(new HTableIncommon(table), FIRST_COLKEY + ":"); - Scanner scanner = - table.getScanner(new Text [] {new Text(FIRST_COLKEY + ":")}, - HConstants.EMPTY_START_ROW, new Text(lastKey)); + byte [][] cols = new byte [1][]; + cols[0] = Bytes.toBytes(FIRST_COLKEY + ":"); + Scanner scanner = table.getScanner(cols, + HConstants.EMPTY_START_ROW, Bytes.toBytes(lastKey)); for (RowResult e: scanner) { if(e.getRow().toString().compareTo(lastKey) >= 0) { LOG.info(e.getRow()); @@ -170,16 +166,15 @@ public class DisabledTestScanner2 extends HBaseClusterTestCase { HBaseAdmin admin = new HBaseAdmin(conf); // Setup colkeys to be inserted - Text tableName = new Text(getName()); - createTable(admin, tableName); - HTable table = new HTable(this.conf, tableName); + createTable(admin, getName()); + HTable table = new HTable(this.conf, getName()); // Add a row to columns without qualifiers and then two with. Make one // numbers only so easy to find w/ a regex. - BatchUpdate batchUpdate = new BatchUpdate(new Text(getName())); + BatchUpdate batchUpdate = new BatchUpdate(getName()); final String firstColkeyFamily = Character.toString(FIRST_COLKEY) + ":"; - batchUpdate.put(new Text(firstColkeyFamily + getName()), GOOD_BYTES); - batchUpdate.put(new Text(firstColkeyFamily + "22222"), GOOD_BYTES); - batchUpdate.put(new Text(firstColkeyFamily), GOOD_BYTES); + batchUpdate.put(firstColkeyFamily + getName(), GOOD_BYTES); + batchUpdate.put(firstColkeyFamily + "22222", GOOD_BYTES); + batchUpdate.put(firstColkeyFamily, GOOD_BYTES); table.commit(batchUpdate); // Now do a scan using a regex for a column name. checkRegexingScanner(table, firstColkeyFamily + "\\d+"); @@ -197,12 +192,12 @@ public class DisabledTestScanner2 extends HBaseClusterTestCase { private void checkRegexingScanner(final HTable table, final String regexColumnname) throws IOException { - Text [] regexCol = new Text [] {new Text(regexColumnname)}; - Scanner scanner = - table.getScanner(regexCol, HConstants.EMPTY_START_ROW); + byte [][] regexCols = new byte[1][]; + regexCols[0] = Bytes.toBytes(regexColumnname); + Scanner scanner = table.getScanner(regexCols, HConstants.EMPTY_START_ROW); int count = 0; for (RowResult r : scanner) { - for (Text c: r.keySet()) { + for (byte [] c: r.keySet()) { System.out.println(c); assertTrue(c.toString().matches(regexColumnname)); count++; @@ -222,14 +217,13 @@ public class DisabledTestScanner2 extends HBaseClusterTestCase { HBaseAdmin admin = new HBaseAdmin(conf); // Setup colkeys to be inserted - Text tableName = new Text(getName()); - Text [] colKeys = createTable(admin, tableName); + byte [][] colKeys = createTable(admin, getName()); assertTrue("Master is running.", admin.isMasterRunning()); // Enter data - HTable table = new HTable(conf, tableName); + HTable table = new HTable(conf, getName()); for (char i = FIRST_ROWKEY; i <= LAST_ROWKEY; i++) { - Text rowKey = new Text(new String(new char[] { i })); + byte [] rowKey = new byte [] { (byte)i }; BatchUpdate batchUpdate = new BatchUpdate(rowKey); for (char j = 0; j < colKeys.length; j++) { batchUpdate.put(colKeys[j], (i >= FIRST_BAD_RANGE_ROWKEY && @@ -248,14 +242,13 @@ public class DisabledTestScanner2 extends HBaseClusterTestCase { * @return Returns column keys used making table. * @throws IOException */ - private Text [] createTable(final HBaseAdmin admin, final Text tableName) + private byte [][] createTable(final HBaseAdmin admin, final String tableName) throws IOException { // Setup colkeys to be inserted HTableDescriptor htd = new HTableDescriptor(getName()); - Text[] colKeys = new Text[(LAST_COLKEY - FIRST_COLKEY) + 1]; + byte [][] colKeys = new byte[(LAST_COLKEY - FIRST_COLKEY) + 1][]; for (char i = 0; i < colKeys.length; i++) { - colKeys[i] = new Text(new String(new char[] { - (char)(FIRST_COLKEY + i), ':' })); + colKeys[i] = new byte [] {(byte)(FIRST_COLKEY + i), ':' }; htd.addFamily(new HColumnDescriptor(colKeys[i].toString())); } admin.createTable(htd); @@ -264,37 +257,38 @@ public class DisabledTestScanner2 extends HBaseClusterTestCase { return colKeys; } - private void regExpFilterTest(HTable table, Text[] colKeys) + private void regExpFilterTest(HTable table, byte [][] colKeys) throws Exception { // Get the filter. The RegExpRowFilter used should filter out vowels. - Map colCriteria = new TreeMap(); + Map colCriteria = + new TreeMap(Bytes.BYTES_COMPARATOR); for (int i = 0; i < colKeys.length; i++) { colCriteria.put(colKeys[i], GOOD_BYTES); } RowFilterInterface filter = new RegExpRowFilter("[^aeiou]", colCriteria); // Create the scanner from the filter. - Scanner scanner = table.getScanner(colKeys, new Text(new - String(new char[] { FIRST_ROWKEY })), filter); + Scanner scanner = table.getScanner(colKeys, new byte [] { FIRST_ROWKEY }, + filter); // Iterate over the scanner, ensuring that results match the passed regex. iterateOnScanner(scanner, "[^aei-qu]"); } - private void rowFilterSetTest(HTable table, Text[] colKeys) + private void rowFilterSetTest(HTable table, byte [][] colKeys) throws Exception { // Get the filter. The RegExpRowFilter used should filter out vowels and // the WhileMatchRowFilter(StopRowFilter) should filter out all rows // greater than or equal to 'r'. Set filterSet = new HashSet(); filterSet.add(new RegExpRowFilter("[^aeiou]")); - filterSet.add(new WhileMatchRowFilter(new StopRowFilter(new Text("r")))); + filterSet.add(new WhileMatchRowFilter(new StopRowFilter(Bytes.toBytes("r")))); RowFilterInterface filter = new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ALL, filterSet); // Create the scanner from the filter. - Scanner scanner = table.getScanner(colKeys, new Text(new - String(new char[] { FIRST_ROWKEY })), filter); + Scanner scanner = table.getScanner(colKeys, new byte [] { FIRST_ROWKEY }, + filter); // Iterate over the scanner, ensuring that results match the passed regex. iterateOnScanner(scanner, "[^aeior-z]"); @@ -327,8 +321,7 @@ public class DisabledTestScanner2 extends HBaseClusterTestCase { HTable metaTable = new HTable(conf, HConstants.META_TABLE_NAME); // First add a new table. Its intial region will be added to META region. HBaseAdmin admin = new HBaseAdmin(conf); - Text tableName = new Text(getName()); - admin.createTable(new HTableDescriptor(tableName.toString())); + admin.createTable(new HTableDescriptor(getName())); List regions = scan(metaTable); assertEquals("Expected one region", 1, regions.size()); HRegionInfo region = regions.get(0); @@ -341,10 +334,10 @@ public class DisabledTestScanner2 extends HBaseClusterTestCase { Path homedir = new Path(getName()); List newRegions = new ArrayList(2); newRegions.add(HRegion.createHRegion( - new HRegionInfo(desc, null, new Text("midway")), + new HRegionInfo(desc, null, Bytes.toBytes("midway")), homedir, this.conf)); newRegions.add(HRegion.createHRegion( - new HRegionInfo(desc, new Text("midway"), null), + new HRegionInfo(desc, Bytes.toBytes("midway"), null), homedir, this.conf)); try { for (HRegion r : newRegions) { @@ -370,8 +363,8 @@ public class DisabledTestScanner2 extends HBaseClusterTestCase { HRegionLocation rl = t.getRegionLocation(t.getTableName()); regionServer = t.getConnection().getHRegionConnection(rl.getServerAddress()); scannerId = regionServer.openScanner(rl.getRegionInfo().getRegionName(), - HConstants.COLUMN_FAMILY_ARRAY, new Text(), - System.currentTimeMillis(), null); + HConstants.COLUMN_FAMILY_ARRAY, HConstants.EMPTY_START_ROW, + HConstants.LATEST_TIMESTAMP, null); while (true) { RowResult values = regionServer.next(scannerId); if (values == null || values.size() == 0) { @@ -414,8 +407,8 @@ public class DisabledTestScanner2 extends HBaseClusterTestCase { batchUpdate.put(HConstants.COL_REGIONINFO, Writables.getBytes(region.getRegionInfo())); batchUpdate.put(HConstants.COL_SERVER, - Writables.stringToBytes(serverAddress.toString())); - batchUpdate.put(HConstants.COL_STARTCODE, Writables.longToBytes(startCode)); + Bytes.toBytes(serverAddress.toString())); + batchUpdate.put(HConstants.COL_STARTCODE, Bytes.toBytes(startCode)); t.commit(batchUpdate); // Assert added. byte [] bytes = @@ -435,7 +428,7 @@ public class DisabledTestScanner2 extends HBaseClusterTestCase { * @param regionName Region to remove. * @throws IOException */ - private void removeRegionFromMETA(final HTable t, final Text regionName) + private void removeRegionFromMETA(final HTable t, final byte [] regionName) throws IOException { BatchUpdate batchUpdate = new BatchUpdate(regionName); batchUpdate.delete(HConstants.COL_REGIONINFO); diff --git a/src/test/org/apache/hadoop/hbase/HBaseClusterTestCase.java b/src/test/org/apache/hadoop/hbase/HBaseClusterTestCase.java index 14c32687630..f31a697859b 100644 --- a/src/test/org/apache/hadoop/hbase/HBaseClusterTestCase.java +++ b/src/test/org/apache/hadoop/hbase/HBaseClusterTestCase.java @@ -30,7 +30,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.io.Text; /** * Abstract base class for HBase cluster junit tests. Spins up an hbase @@ -80,6 +79,7 @@ public abstract class HBaseClusterTestCase extends HBaseTestCase { * Run after dfs is ready but before hbase cluster is started up. */ protected void preHBaseClusterSetup() throws Exception { + // continue } /** @@ -90,13 +90,14 @@ public abstract class HBaseClusterTestCase extends HBaseTestCase { // start the mini cluster this.cluster = new MiniHBaseCluster(conf, regionServers); // opening the META table ensures that cluster is running - HTable meta = new HTable(conf, new Text(".META.")); + HTable meta = new HTable(conf, ".META."); } /** * Run after hbase cluster is started up. */ protected void postHBaseClusterSetup() throws Exception { + // continue } @Override diff --git a/src/test/org/apache/hadoop/hbase/HBaseTestCase.java b/src/test/org/apache/hadoop/hbase/HBaseTestCase.java index 4af4f6f1e8c..2ae952d8ac8 100644 --- a/src/test/org/apache/hadoop/hbase/HBaseTestCase.java +++ b/src/test/org/apache/hadoop/hbase/HBaseTestCase.java @@ -34,13 +34,13 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType; import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Scanner; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.util.Bytes; /** * Abstract base class for test cases. Performs all static initialization @@ -50,13 +50,13 @@ public abstract class HBaseTestCase extends TestCase { /** configuration parameter name for test directory */ public static final String TEST_DIRECTORY_KEY = "test.build.data"; - - protected final static String COLFAMILY_NAME1 = "colfamily1:"; - protected final static String COLFAMILY_NAME2 = "colfamily2:"; - protected final static String COLFAMILY_NAME3 = "colfamily3:"; - protected static Text [] COLUMNS = new Text [] {new Text(COLFAMILY_NAME1), - new Text(COLFAMILY_NAME2), new Text(COLFAMILY_NAME3)}; + protected final static byte [] COLFAMILY_NAME1 = Bytes.toBytes("colfamily1:"); + protected final static byte [] COLFAMILY_NAME2 = Bytes.toBytes("colfamily2:"); + protected final static byte [] COLFAMILY_NAME3 = Bytes.toBytes("colfamily3:"); + protected static final byte [][] COLUMNS = {COLFAMILY_NAME1, + COLFAMILY_NAME2, COLFAMILY_NAME3}; + private boolean localfs = false; protected Path testDir = null; protected FileSystem fs = null; @@ -151,9 +151,9 @@ public abstract class HBaseTestCase extends TestCase { conf.get(TEST_DIRECTORY_KEY, "test/build/data"), testName); } - protected HRegion createNewHRegion(HTableDescriptor desc, Text startKey, - Text endKey) throws IOException { - + protected HRegion createNewHRegion(HTableDescriptor desc, byte [] startKey, + byte [] endKey) + throws IOException { FileSystem filesystem = FileSystem.get(conf); Path rootdir = filesystem.makeQualified( new Path(conf.get(HConstants.HBASE_DIR))); @@ -190,13 +190,13 @@ public abstract class HBaseTestCase extends TestCase { protected HTableDescriptor createTableDescriptor(final String name, final int versions) { HTableDescriptor htd = new HTableDescriptor(name); - htd.addFamily(new HColumnDescriptor(new Text(COLFAMILY_NAME1), versions, + htd.addFamily(new HColumnDescriptor(COLFAMILY_NAME1, versions, CompressionType.NONE, false, false, Integer.MAX_VALUE, HConstants.FOREVER, null)); - htd.addFamily(new HColumnDescriptor(new Text(COLFAMILY_NAME2), versions, + htd.addFamily(new HColumnDescriptor(COLFAMILY_NAME2, versions, CompressionType.NONE, false, false, Integer.MAX_VALUE, HConstants.FOREVER, null)); - htd.addFamily(new HColumnDescriptor(new Text(COLFAMILY_NAME3), versions, + htd.addFamily(new HColumnDescriptor(COLFAMILY_NAME3, versions, CompressionType.NONE, false, false, Integer.MAX_VALUE, HConstants.FOREVER, null)); return htd; @@ -210,15 +210,16 @@ public abstract class HBaseTestCase extends TestCase { * @param column * @throws IOException */ - protected static void addContent(final HRegion r, final String column) + protected static void addContent(final HRegion r, final byte [] column) throws IOException { - Text startKey = r.getRegionInfo().getStartKey(); - Text endKey = r.getRegionInfo().getEndKey(); - byte [] startKeyBytes = startKey.getBytes(); + byte [] startKey = r.getRegionInfo().getStartKey(); + byte [] endKey = r.getRegionInfo().getEndKey(); + byte [] startKeyBytes = startKey; if (startKeyBytes == null || startKeyBytes.length == 0) { startKeyBytes = START_KEY_BYTES; } - addContent(new HRegionIncommon(r), column, startKeyBytes, endKey, -1); + addContent(new HRegionIncommon(r), Bytes.toString(column), + startKeyBytes, endKey, -1); } /** @@ -245,7 +246,7 @@ public abstract class HBaseTestCase extends TestCase { * @throws IOException */ protected static void addContent(final Incommon updater, final String column, - final byte [] startKeyBytes, final Text endKey) + final byte [] startKeyBytes, final byte [] endKey) throws IOException { addContent(updater, column, startKeyBytes, endKey, -1); } @@ -262,7 +263,7 @@ public abstract class HBaseTestCase extends TestCase { * @throws IOException */ protected static void addContent(final Incommon updater, final String column, - final byte [] startKeyBytes, final Text endKey, final long ts) + final byte [] startKeyBytes, final byte [] endKey, final long ts) throws IOException { // Add rows of three characters. The first character starts with the // 'a' character and runs up to 'z'. Per first character, we run the @@ -274,18 +275,17 @@ 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}; - 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) { + String s = Bytes.toString(bytes) + PUNCTUATION; + byte [] t = Bytes.toBytes(s); + if (endKey != null && endKey.length > 0 + && Bytes.compareTo(endKey, t) <= 0) { break EXIT; } try { BatchUpdate batchUpdate = ts == -1 ? new BatchUpdate(t) : new BatchUpdate(t, ts); try { - batchUpdate.put(new Text(column), bytes); + batchUpdate.put(column, t); updater.commit(batchUpdate); } catch (RuntimeException ex) { ex.printStackTrace(); @@ -332,7 +332,7 @@ public abstract class HBaseTestCase extends TestCase { * @return value for row/column pair * @throws IOException */ - public Cell get(Text row, Text column) throws IOException; + public Cell get(byte [] row, byte [] column) throws IOException; /** * @param row * @param column @@ -340,7 +340,7 @@ public abstract class HBaseTestCase extends TestCase { * @return value for row/column pair for number of versions requested * @throws IOException */ - public Cell[] get(Text row, Text column, int versions) throws IOException; + public Cell[] get(byte [] row, byte [] column, int versions) throws IOException; /** * @param row * @param column @@ -349,7 +349,7 @@ public abstract class HBaseTestCase extends TestCase { * @return value for row/column/timestamp tuple for number of versions * @throws IOException */ - public Cell[] get(Text row, Text column, long ts, int versions) + public Cell[] get(byte [] row, byte [] column, long ts, int versions) throws IOException; /** * @param row @@ -357,7 +357,7 @@ public abstract class HBaseTestCase extends TestCase { * @param ts * @throws IOException */ - public void deleteAll(Text row, Text column, long ts) throws IOException; + public void deleteAll(byte [] row, byte [] column, long ts) throws IOException; /** * @param batchUpdate @@ -372,7 +372,7 @@ public abstract class HBaseTestCase extends TestCase { * @return scanner for specified columns, first row and timestamp * @throws IOException */ - public ScannerIncommon getScanner(Text [] columns, Text firstRow, + public ScannerIncommon getScanner(byte [] [] columns, byte [] firstRow, long ts) throws IOException; } @@ -403,29 +403,32 @@ public abstract class HBaseTestCase extends TestCase { }; /** {@inheritDoc} */ - public void deleteAll(Text row, Text column, long ts) throws IOException { + public void deleteAll(byte [] row, byte [] column, long ts) + throws IOException { this.region.deleteAll(row, column, ts); } /** {@inheritDoc} */ - public ScannerIncommon getScanner(Text [] columns, Text firstRow, long ts) + public ScannerIncommon getScanner(byte [][] columns, byte [] firstRow, + long ts) throws IOException { return new InternalScannerIncommon(region.getScanner(columns, firstRow, ts, null)); } /** {@inheritDoc} */ - public Cell get(Text row, Text column) throws IOException { + public Cell get(byte [] row, byte [] column) throws IOException { return this.region.get(row, column); } /** {@inheritDoc} */ - public Cell[] get(Text row, Text column, int versions) throws IOException { + public Cell[] get(byte [] row, byte [] column, int versions) + throws IOException { return this.region.get(row, column, versions); } /** {@inheritDoc} */ - public Cell[] get(Text row, Text column, long ts, int versions) + public Cell[] get(byte [] row, byte [] column, long ts, int versions) throws IOException { return this.region.get(row, column, ts, versions); } @@ -435,7 +438,7 @@ public abstract class HBaseTestCase extends TestCase { * @return values for each column in the specified row * @throws IOException */ - public Map getFull(Text row) throws IOException { + public Map getFull(byte [] row) throws IOException { return region.getFull(row, null, HConstants.LATEST_TIMESTAMP); } @@ -473,37 +476,39 @@ public abstract class HBaseTestCase extends TestCase { }; /** {@inheritDoc} */ - public void deleteAll(Text row, Text column, long ts) throws IOException { + public void deleteAll(byte [] row, byte [] column, long ts) + throws IOException { this.table.deleteAll(row, column, ts); } /** {@inheritDoc} */ - public ScannerIncommon getScanner(Text [] columns, Text firstRow, long ts) + public ScannerIncommon getScanner(byte [][] columns, byte [] firstRow, long ts) throws IOException { return new ClientScannerIncommon(table.getScanner(columns, firstRow, ts, null)); } /** {@inheritDoc} */ - public Cell get(Text row, Text column) throws IOException { + public Cell get(byte [] row, byte [] column) throws IOException { return this.table.get(row, column); } /** {@inheritDoc} */ - public Cell[] get(Text row, Text column, int versions) throws IOException { + public Cell[] get(byte [] row, byte [] column, int versions) + throws IOException { return this.table.get(row, column, versions); } /** {@inheritDoc} */ - public Cell[] get(Text row, Text column, long ts, int versions) + public Cell[] get(byte [] row, byte [] column, long ts, int versions) throws IOException { return this.table.get(row, column, ts, versions); } } public interface ScannerIncommon - extends Iterable>> { - public boolean next(HStoreKey key, SortedMap values) + extends Iterable>> { + public boolean next(HStoreKey key, SortedMap values) throws IOException; public void close() throws IOException; @@ -515,19 +520,18 @@ public abstract class HBaseTestCase extends TestCase { this.scanner = scanner; } - public boolean next(HStoreKey key, SortedMap values) + public boolean next(HStoreKey key, SortedMap values) throws IOException { RowResult results = scanner.next(); if (results == null) { return false; - } else { - key.setRow(results.getRow()); - values.clear(); - for (Map.Entry entry : results.entrySet()) { - values.put(entry.getKey(), entry.getValue().getValue()); - } - return true; } + key.setRow(results.getRow()); + values.clear(); + for (Map.Entry entry : results.entrySet()) { + values.put(entry.getKey(), entry.getValue().getValue()); + } + return true; } public void close() throws IOException { @@ -546,7 +550,7 @@ public abstract class HBaseTestCase extends TestCase { this.scanner = scanner; } - public boolean next(HStoreKey key, SortedMap values) + public boolean next(HStoreKey key, SortedMap values) throws IOException { return scanner.next(key, values); } @@ -560,10 +564,10 @@ public abstract class HBaseTestCase extends TestCase { } } - protected void assertCellEquals(final HRegion region, final Text row, - final Text column, final long timestamp, final String value) + protected void assertCellEquals(final HRegion region, final byte [] row, + final byte [] column, final long timestamp, final String value) throws IOException { - Map result = region.getFull(row, null, timestamp); + Map result = region.getFull(row, null, timestamp); Cell cell_value = result.get(column); if(value == null){ assertEquals(column.toString() + " at timestamp " + timestamp, null, cell_value); diff --git a/src/test/org/apache/hadoop/hbase/MapFilePerformanceEvaluation.java b/src/test/org/apache/hadoop/hbase/MapFilePerformanceEvaluation.java index b78787a6f2f..e92db64223b 100644 --- a/src/test/org/apache/hadoop/hbase/MapFilePerformanceEvaluation.java +++ b/src/test/org/apache/hadoop/hbase/MapFilePerformanceEvaluation.java @@ -27,8 +27,9 @@ import org.apache.commons.math.random.RandomDataImpl; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.MapFile; -import org.apache.hadoop.io.Text; import org.apache.log4j.Logger; /** @@ -44,10 +45,10 @@ public class MapFilePerformanceEvaluation { static final Logger LOG = Logger.getLogger(MapFilePerformanceEvaluation.class.getName()); - static Text format(final int i, final Text text) { + static ImmutableBytesWritable format(final int i, ImmutableBytesWritable w) { String v = Integer.toString(i); - text.set("0000000000".substring(v.length()) + v); - return text; + w.set(Bytes.toBytes("0000000000".substring(v.length()) + v)); + return w; } private void runBenchmarks() throws Exception { @@ -84,8 +85,6 @@ public class MapFilePerformanceEvaluation { protected final FileSystem fs; protected final Path mf; protected final int totalRows; - protected Text key; - protected Text val; public RowOrientedBenchmark(Configuration conf, FileSystem fs, Path mf, int totalRows) { @@ -93,8 +92,6 @@ public class MapFilePerformanceEvaluation { this.fs = fs; this.mf = mf; this.totalRows = totalRows; - this.key = new Text(); - this.val = new Text(); } void setUp() throws Exception { @@ -141,6 +138,8 @@ public class MapFilePerformanceEvaluation { protected MapFile.Writer writer; private Random random = new Random(); private byte[] bytes = new byte[ROW_LENGTH]; + private ImmutableBytesWritable key = new ImmutableBytesWritable(); + private ImmutableBytesWritable value = new ImmutableBytesWritable(); public SequentialWriteBenchmark(Configuration conf, FileSystem fs, Path mf, int totalRows) { @@ -150,13 +149,13 @@ public class MapFilePerformanceEvaluation { @Override void setUp() throws Exception { writer = new MapFile.Writer(conf, fs, mf.toString(), - Text.class, Text.class); + ImmutableBytesWritable.class, ImmutableBytesWritable.class); } @Override void doRow(int i) throws Exception { - val.set(generateValue()); - writer.append(format(i, key), val); + value.set(generateValue()); + writer.append(format(i, key), value); } private byte[] generateValue() { @@ -177,6 +176,8 @@ public class MapFilePerformanceEvaluation { } static abstract class ReadBenchmark extends RowOrientedBenchmark { + ImmutableBytesWritable key = new ImmutableBytesWritable(); + ImmutableBytesWritable value = new ImmutableBytesWritable(); protected MapFile.Reader reader; @@ -198,7 +199,7 @@ public class MapFilePerformanceEvaluation { } static class SequentialReadBenchmark extends ReadBenchmark { - + public SequentialReadBenchmark(Configuration conf, FileSystem fs, Path mf, int totalRows) { super(conf, fs, mf, totalRows); @@ -206,7 +207,7 @@ public class MapFilePerformanceEvaluation { @Override void doRow(@SuppressWarnings("unused") int i) throws Exception { - reader.next(key, val); + reader.next(key, value); } @Override @@ -227,10 +228,10 @@ public class MapFilePerformanceEvaluation { @Override void doRow(@SuppressWarnings("unused") int i) throws Exception { - reader.get(getRandomRow(), val); + reader.get(getRandomRow(), value); } - private Text getRandomRow() { + private ImmutableBytesWritable getRandomRow() { return format(random.nextInt(totalRows), key); } @@ -247,10 +248,10 @@ public class MapFilePerformanceEvaluation { @Override void doRow(@SuppressWarnings("unused") int i) throws Exception { - reader.get(getGaussianRandomRow(), val); + reader.get(getGaussianRandomRow(), value); } - private Text getGaussianRandomRow() { + private ImmutableBytesWritable getGaussianRandomRow() { int r = (int) randomData.nextGaussian(totalRows / 2, totalRows / 10); return format(r, key); } diff --git a/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java b/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java index 9cf42749e51..670cde6e121 100644 --- a/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java +++ b/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java @@ -149,7 +149,7 @@ public class MiniHBaseCluster implements HConstants { public void flushcache() throws IOException { for (LocalHBaseCluster.RegionServerThread t: this.hbaseCluster.getRegionServers()) { - for(HRegion r: t.getRegionServer().getOnlineRegions().values() ) { + for(HRegion r: t.getRegionServer().getOnlineRegions()) { r.flushcache(); } } diff --git a/src/test/org/apache/hadoop/hbase/MultiRegionTable.java b/src/test/org/apache/hadoop/hbase/MultiRegionTable.java index 864d8fbbf64..7c65c568106 100644 --- a/src/test/org/apache/hadoop/hbase/MultiRegionTable.java +++ b/src/test/org/apache/hadoop/hbase/MultiRegionTable.java @@ -21,43 +21,42 @@ package org.apache.hadoop.hbase; import java.io.IOException; -import org.apache.hadoop.io.Text; - import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.util.Bytes; /** * Utility class to build a table of multiple regions. */ public class MultiRegionTable extends HBaseClusterTestCase { - private static final Text[] KEYS = { - null, - new Text("bbb"), - new Text("ccc"), - new Text("ddd"), - new Text("eee"), - new Text("fff"), - new Text("ggg"), - new Text("hhh"), - new Text("iii"), - new Text("jjj"), - new Text("kkk"), - new Text("lll"), - new Text("mmm"), - new Text("nnn"), - new Text("ooo"), - new Text("ppp"), - new Text("qqq"), - new Text("rrr"), - new Text("sss"), - new Text("ttt"), - new Text("uuu"), - new Text("vvv"), - new Text("www"), - new Text("xxx"), - new Text("yyy") + private static final byte [][] KEYS = { + HConstants.EMPTY_BYTE_ARRAY, + Bytes.toBytes("bbb"), + Bytes.toBytes("ccc"), + Bytes.toBytes("ddd"), + Bytes.toBytes("eee"), + Bytes.toBytes("fff"), + Bytes.toBytes("ggg"), + Bytes.toBytes("hhh"), + Bytes.toBytes("iii"), + Bytes.toBytes("jjj"), + Bytes.toBytes("kkk"), + Bytes.toBytes("lll"), + Bytes.toBytes("mmm"), + Bytes.toBytes("nnn"), + Bytes.toBytes("ooo"), + Bytes.toBytes("ppp"), + Bytes.toBytes("qqq"), + Bytes.toBytes("rrr"), + Bytes.toBytes("sss"), + Bytes.toBytes("ttt"), + Bytes.toBytes("uuu"), + Bytes.toBytes("vvv"), + Bytes.toBytes("www"), + Bytes.toBytes("xxx"), + Bytes.toBytes("yyy") }; - protected final String columnName; + protected final byte [] columnName; protected HTableDescriptor desc; /** @@ -65,7 +64,7 @@ public class MultiRegionTable extends HBaseClusterTestCase { */ public MultiRegionTable(final String columnName) { super(); - this.columnName = columnName; + this.columnName = Bytes.toBytes(columnName); // These are needed for the new and improved Map/Reduce framework System.setProperty("hadoop.log.dir", conf.get("hadoop.log.dir")); conf.set("mapred.output.dir", conf.get("hadoop.tmp.dir")); @@ -78,7 +77,6 @@ public class MultiRegionTable extends HBaseClusterTestCase { protected void preHBaseClusterSetup() throws Exception { try { // Create a bunch of regions - HRegion[] regions = new HRegion[KEYS.length]; for (int i = 0; i < regions.length; i++) { int j = (i + 1) % regions.length; @@ -87,17 +85,14 @@ public class MultiRegionTable extends HBaseClusterTestCase { // Now create the root and meta regions and insert the data regions // created above into the meta - - HRegion root = HRegion.createHRegion(HRegionInfo.rootRegionInfo, + HRegion root = HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO, testDir, this.conf); - HRegion meta = HRegion.createHRegion(HRegionInfo.firstMetaRegionInfo, + HRegion meta = HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO, testDir, this.conf); HRegion.addRegionToMETA(root, meta); - for(int i = 0; i < regions.length; i++) { HRegion.addRegionToMETA(meta, regions[i]); } - closeRegionAndDeleteLog(root); closeRegionAndDeleteLog(meta); } catch (Exception e) { @@ -106,7 +101,7 @@ public class MultiRegionTable extends HBaseClusterTestCase { } } - private HRegion createARegion(Text startKey, Text endKey) throws IOException { + private HRegion createARegion(byte [] startKey, byte [] endKey) throws IOException { HRegion region = createNewHRegion(desc, startKey, endKey); addContent(region, this.columnName); closeRegionAndDeleteLog(region); diff --git a/src/test/org/apache/hadoop/hbase/PerformanceEvaluation.java b/src/test/org/apache/hadoop/hbase/PerformanceEvaluation.java index dcdef6dbfc2..5fca2e84b12 100644 --- a/src/test/org/apache/hadoop/hbase/PerformanceEvaluation.java +++ b/src/test/org/apache/hadoop/hbase/PerformanceEvaluation.java @@ -26,19 +26,21 @@ import java.util.Arrays; import java.util.Date; import java.util.List; import java.util.Random; -import java.util.TreeMap; import java.util.regex.Matcher; import java.util.regex.Pattern; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.dfs.MiniDFSCluster; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Scanner; +import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.MapReduceBase; @@ -48,10 +50,6 @@ import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.TextInputFormat; import org.apache.hadoop.mapred.TextOutputFormat; import org.apache.log4j.Logger; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.Scanner; -import org.apache.hadoop.hbase.io.BatchUpdate; /** @@ -77,12 +75,12 @@ public class PerformanceEvaluation implements HConstants { private static final int ROW_LENGTH = 1000; private static final int ONE_GB = 1024 * 1024 * 1000; private static final int ROWS_PER_GB = ONE_GB / ROW_LENGTH; - static final Text COLUMN_NAME = new Text(COLUMN_FAMILY + "data"); + static final byte [] COLUMN_NAME = Bytes.toBytes(COLUMN_FAMILY_STR + "data"); protected static HTableDescriptor tableDescriptor; static { tableDescriptor = new HTableDescriptor("TestTable"); - tableDescriptor.addFamily(new HColumnDescriptor(COLUMN_FAMILY.toString())); + tableDescriptor.addFamily(new HColumnDescriptor(COLUMN_FAMILY)); } private static final String RANDOM_READ = "randomRead"; @@ -388,7 +386,7 @@ public class PerformanceEvaluation implements HConstants { @Override void testRow(@SuppressWarnings("unused") final int i) throws IOException { - Text row = getRandomRow(this.rand, this.totalRows); + byte [] row = getRandomRow(this.rand, this.totalRows); BatchUpdate b = new BatchUpdate(row); b.put(COLUMN_NAME, generateValue(this.rand)); table.commit(b); @@ -411,7 +409,7 @@ public class PerformanceEvaluation implements HConstants { @Override void testSetup() throws IOException { super.testSetup(); - this.testScanner = table.getScanner(new Text[] {COLUMN_NAME}, + this.testScanner = table.getScanner(new byte [][] {COLUMN_NAME}, format(this.startRow)); } @@ -473,14 +471,18 @@ public class PerformanceEvaluation implements HConstants { /* * Format passed integer. - * This method takes some time and is done inline uploading data. For - * example, doing the mapfile test, generation of the key and value - * consumes about 30% of CPU time. - * @param i - * @return Integer as String zero padded. + * @param number + * @return Returns zero-prefixed 10-byte wide decimal version of passed + * number (Does absolute in case number is negative). */ - static Text format(final int i) { - return new Text(String.format("%010d", Integer.valueOf(i))); + static byte [] format(final int number) { + byte [] b = new byte[10]; + int d = Math.abs(number); + for (int i = b.length - 1; i > 0; i--) { + b[i] = (byte)((d % 10) + '0'); + d /= 10; + } + return b; } /* @@ -495,8 +497,8 @@ public class PerformanceEvaluation implements HConstants { return b; } - static Text getRandomRow(final Random random, final int totalRows) { - return new Text(format(random.nextInt(Integer.MAX_VALUE) % totalRows)); + static byte [] getRandomRow(final Random random, final int totalRows) { + return format(random.nextInt(Integer.MAX_VALUE) % totalRows); } long runOneClient(final String cmd, final int startRow, @@ -695,7 +697,8 @@ public class PerformanceEvaluation implements HConstants { * @param args */ public static void main(final String[] args) { - System.exit(new PerformanceEvaluation(new HBaseConfiguration()). + HBaseConfiguration c = new HBaseConfiguration(); + System.exit(new PerformanceEvaluation(c). doCommandLine(args)); } -} +} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/TestBloomFilters.java b/src/test/org/apache/hadoop/hbase/TestBloomFilters.java index 4a0dcc584b4..b9edc7703be 100644 --- a/src/test/org/apache/hadoop/hbase/TestBloomFilters.java +++ b/src/test/org/apache/hadoop/hbase/TestBloomFilters.java @@ -22,127 +22,127 @@ package org.apache.hadoop.hbase; import java.io.IOException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.util.Bytes; /** Tests per-column bloom filters */ public class TestBloomFilters extends HBaseClusterTestCase { static final Log LOG = LogFactory.getLog(TestBloomFilters.class); - private static final Text CONTENTS = new Text("contents:"); + private static final byte [] CONTENTS = Bytes.toBytes("contents:"); - private static final Text[] rows = { - new Text("wmjwjzyv"), - new Text("baietibz"), - new Text("guhsgxnv"), - new Text("mhnqycto"), - new Text("xcyqafgz"), - new Text("zidoamgb"), - new Text("tftfirzd"), - new Text("okapqlrg"), - new Text("yccwzwsq"), - new Text("qmonufqu"), - new Text("wlsctews"), - new Text("mksdhqri"), - new Text("wxxllokj"), - new Text("eviuqpls"), - new Text("bavotqmj"), - new Text("yibqzhdl"), - new Text("csfqmsyr"), - new Text("guxliyuh"), - new Text("pzicietj"), - new Text("qdwgrqwo"), - new Text("ujfzecmi"), - new Text("dzeqfvfi"), - new Text("phoegsij"), - new Text("bvudfcou"), - new Text("dowzmciz"), - new Text("etvhkizp"), - new Text("rzurqycg"), - new Text("krqfxuge"), - new Text("gflcohtd"), - new Text("fcrcxtps"), - new Text("qrtovxdq"), - new Text("aypxwrwi"), - new Text("dckpyznr"), - new Text("mdaawnpz"), - new Text("pakdfvca"), - new Text("xjglfbez"), - new Text("xdsecofi"), - new Text("sjlrfcab"), - new Text("ebcjawxv"), - new Text("hkafkjmy"), - new Text("oimmwaxo"), - new Text("qcuzrazo"), - new Text("nqydfkwk"), - new Text("frybvmlb"), - new Text("amxmaqws"), - new Text("gtkovkgx"), - new Text("vgwxrwss"), - new Text("xrhzmcep"), - new Text("tafwziil"), - new Text("erjmncnv"), - new Text("heyzqzrn"), - new Text("sowvyhtu"), - new Text("heeixgzy"), - new Text("ktcahcob"), - new Text("ljhbybgg"), - new Text("jiqfcksl"), - new Text("anjdkjhm"), - new Text("uzcgcuxp"), - new Text("vzdhjqla"), - new Text("svhgwwzq"), - new Text("zhswvhbp"), - new Text("ueceybwy"), - new Text("czkqykcw"), - new Text("ctisayir"), - new Text("hppbgciu"), - new Text("nhzgljfk"), - new Text("vaziqllf"), - new Text("narvrrij"), - new Text("kcevbbqi"), - new Text("qymuaqnp"), - new Text("pwqpfhsr"), - new Text("peyeicuk"), - new Text("kudlwihi"), - new Text("pkmqejlm"), - new Text("ylwzjftl"), - new Text("rhqrlqar"), - new Text("xmftvzsp"), - new Text("iaemtihk"), - new Text("ymsbrqcu"), - new Text("yfnlcxto"), - new Text("nluqopqh"), - new Text("wmrzhtox"), - new Text("qnffhqbl"), - new Text("zypqpnbw"), - new Text("oiokhatd"), - new Text("mdraddiu"), - new Text("zqoatltt"), - new Text("ewhulbtm"), - new Text("nmswpsdf"), - new Text("xsjeteqe"), - new Text("ufubcbma"), - new Text("phyxvrds"), - new Text("vhnfldap"), - new Text("zrrlycmg"), - new Text("becotcjx"), - new Text("wvbubokn"), - new Text("avkgiopr"), - new Text("mbqqxmrv"), - new Text("ibplgvuu"), - new Text("dghvpkgc") + private static final byte [][] rows = { + Bytes.toBytes("wmjwjzyv"), + Bytes.toBytes("baietibz"), + Bytes.toBytes("guhsgxnv"), + Bytes.toBytes("mhnqycto"), + Bytes.toBytes("xcyqafgz"), + Bytes.toBytes("zidoamgb"), + Bytes.toBytes("tftfirzd"), + Bytes.toBytes("okapqlrg"), + Bytes.toBytes("yccwzwsq"), + Bytes.toBytes("qmonufqu"), + Bytes.toBytes("wlsctews"), + Bytes.toBytes("mksdhqri"), + Bytes.toBytes("wxxllokj"), + Bytes.toBytes("eviuqpls"), + Bytes.toBytes("bavotqmj"), + Bytes.toBytes("yibqzhdl"), + Bytes.toBytes("csfqmsyr"), + Bytes.toBytes("guxliyuh"), + Bytes.toBytes("pzicietj"), + Bytes.toBytes("qdwgrqwo"), + Bytes.toBytes("ujfzecmi"), + Bytes.toBytes("dzeqfvfi"), + Bytes.toBytes("phoegsij"), + Bytes.toBytes("bvudfcou"), + Bytes.toBytes("dowzmciz"), + Bytes.toBytes("etvhkizp"), + Bytes.toBytes("rzurqycg"), + Bytes.toBytes("krqfxuge"), + Bytes.toBytes("gflcohtd"), + Bytes.toBytes("fcrcxtps"), + Bytes.toBytes("qrtovxdq"), + Bytes.toBytes("aypxwrwi"), + Bytes.toBytes("dckpyznr"), + Bytes.toBytes("mdaawnpz"), + Bytes.toBytes("pakdfvca"), + Bytes.toBytes("xjglfbez"), + Bytes.toBytes("xdsecofi"), + Bytes.toBytes("sjlrfcab"), + Bytes.toBytes("ebcjawxv"), + Bytes.toBytes("hkafkjmy"), + Bytes.toBytes("oimmwaxo"), + Bytes.toBytes("qcuzrazo"), + Bytes.toBytes("nqydfkwk"), + Bytes.toBytes("frybvmlb"), + Bytes.toBytes("amxmaqws"), + Bytes.toBytes("gtkovkgx"), + Bytes.toBytes("vgwxrwss"), + Bytes.toBytes("xrhzmcep"), + Bytes.toBytes("tafwziil"), + Bytes.toBytes("erjmncnv"), + Bytes.toBytes("heyzqzrn"), + Bytes.toBytes("sowvyhtu"), + Bytes.toBytes("heeixgzy"), + Bytes.toBytes("ktcahcob"), + Bytes.toBytes("ljhbybgg"), + Bytes.toBytes("jiqfcksl"), + Bytes.toBytes("anjdkjhm"), + Bytes.toBytes("uzcgcuxp"), + Bytes.toBytes("vzdhjqla"), + Bytes.toBytes("svhgwwzq"), + Bytes.toBytes("zhswvhbp"), + Bytes.toBytes("ueceybwy"), + Bytes.toBytes("czkqykcw"), + Bytes.toBytes("ctisayir"), + Bytes.toBytes("hppbgciu"), + Bytes.toBytes("nhzgljfk"), + Bytes.toBytes("vaziqllf"), + Bytes.toBytes("narvrrij"), + Bytes.toBytes("kcevbbqi"), + Bytes.toBytes("qymuaqnp"), + Bytes.toBytes("pwqpfhsr"), + Bytes.toBytes("peyeicuk"), + Bytes.toBytes("kudlwihi"), + Bytes.toBytes("pkmqejlm"), + Bytes.toBytes("ylwzjftl"), + Bytes.toBytes("rhqrlqar"), + Bytes.toBytes("xmftvzsp"), + Bytes.toBytes("iaemtihk"), + Bytes.toBytes("ymsbrqcu"), + Bytes.toBytes("yfnlcxto"), + Bytes.toBytes("nluqopqh"), + Bytes.toBytes("wmrzhtox"), + Bytes.toBytes("qnffhqbl"), + Bytes.toBytes("zypqpnbw"), + Bytes.toBytes("oiokhatd"), + Bytes.toBytes("mdraddiu"), + Bytes.toBytes("zqoatltt"), + Bytes.toBytes("ewhulbtm"), + Bytes.toBytes("nmswpsdf"), + Bytes.toBytes("xsjeteqe"), + Bytes.toBytes("ufubcbma"), + Bytes.toBytes("phyxvrds"), + Bytes.toBytes("vhnfldap"), + Bytes.toBytes("zrrlycmg"), + Bytes.toBytes("becotcjx"), + Bytes.toBytes("wvbubokn"), + Bytes.toBytes("avkgiopr"), + Bytes.toBytes("mbqqxmrv"), + Bytes.toBytes("ibplgvuu"), + Bytes.toBytes("dghvpkgc") }; - private static final Text[] testKeys = { - new Text("abcdefgh"), - new Text("ijklmnop"), - new Text("qrstuvwx"), - new Text("yzabcdef") + private static final byte [][] testKeys = { + Bytes.toBytes("abcdefgh"), + Bytes.toBytes("ijklmnop"), + Bytes.toBytes("qrstuvwx"), + Bytes.toBytes("yzabcdef") }; /** constructor */ @@ -191,7 +191,7 @@ public class TestBloomFilters extends HBaseClusterTestCase { // Store some values for(int i = 0; i < 100; i++) { - Text row = rows[i]; + byte [] row = rows[i]; String value = row.toString(); BatchUpdate b = new BatchUpdate(row); b.put(CONTENTS, value.getBytes(HConstants.UTF8_ENCODING)); @@ -201,7 +201,6 @@ public class TestBloomFilters extends HBaseClusterTestCase { // Give cache flusher and log roller a chance to run // Otherwise we'll never hit the bloom filter, just the memcache Thread.sleep(conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000) * 2); - } catch (InterruptedException e) { // ignore } @@ -258,7 +257,7 @@ public class TestBloomFilters extends HBaseClusterTestCase { // Store some values for(int i = 0; i < 100; i++) { - Text row = rows[i]; + byte [] row = rows[i]; String value = row.toString(); BatchUpdate b = new BatchUpdate(row); b.put(CONTENTS, value.getBytes(HConstants.UTF8_ENCODING)); diff --git a/src/test/org/apache/hadoop/hbase/TestCompare.java b/src/test/org/apache/hadoop/hbase/TestCompare.java index 3b0e8d22e15..23a084b53ab 100644 --- a/src/test/org/apache/hadoop/hbase/TestCompare.java +++ b/src/test/org/apache/hadoop/hbase/TestCompare.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.util.Bytes; import junit.framework.TestCase; @@ -35,7 +35,7 @@ public class TestCompare extends TestCase { */ public void testHStoreKey() { long timestamp = System.currentTimeMillis(); - Text a = new Text("a"); + byte [] a = Bytes.toBytes("a"); HStoreKey past = new HStoreKey(a, a, timestamp - 10); HStoreKey now = new HStoreKey(a, a, timestamp); HStoreKey future = new HStoreKey(a, a, timestamp + 10); @@ -52,21 +52,21 @@ public class TestCompare extends TestCase { HRegionInfo b = new HRegionInfo(new HTableDescriptor("b"), null, null); assertTrue(a.compareTo(b) != 0); HTableDescriptor t = new HTableDescriptor("t"); - Text midway = new Text("midway"); + byte [] midway = Bytes.toBytes("midway"); 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(t, new Text("a"), new Text("d")); - b = new HRegionInfo(t, new Text("e"), new Text("g")); + a = new HRegionInfo(t, Bytes.toBytes("a"), Bytes.toBytes("d")); + b = new HRegionInfo(t, Bytes.toBytes("e"), Bytes.toBytes("g")); assertTrue(a.compareTo(b) < 0); - a = new HRegionInfo(t, new Text("aaaa"), new Text("dddd")); - b = new HRegionInfo(t, new Text("e"), new Text("g")); + a = new HRegionInfo(t, Bytes.toBytes("aaaa"), Bytes.toBytes("dddd")); + b = new HRegionInfo(t, Bytes.toBytes("e"), Bytes.toBytes("g")); assertTrue(a.compareTo(b) < 0); - a = new HRegionInfo(t, new Text("aaaa"), new Text("dddd")); - b = new HRegionInfo(t, new Text("aaaa"), new Text("eeee")); + a = new HRegionInfo(t, Bytes.toBytes("aaaa"), Bytes.toBytes("dddd")); + b = new HRegionInfo(t, Bytes.toBytes("aaaa"), Bytes.toBytes("eeee")); assertTrue(a.compareTo(b) < 0); } -} +} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/TestEmptyMetaInfo.java b/src/test/org/apache/hadoop/hbase/TestEmptyMetaInfo.java index 3b40709b598..cf0af0a45ec 100644 --- a/src/test/org/apache/hadoop/hbase/TestEmptyMetaInfo.java +++ b/src/test/org/apache/hadoop/hbase/TestEmptyMetaInfo.java @@ -22,16 +22,11 @@ package org.apache.hadoop.hbase; import java.io.IOException; -import java.util.SortedMap; -import java.util.TreeMap; - -import org.apache.hadoop.io.Text; - import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Scanner; -import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.util.Bytes; /** * Tests master cleanup of rows in meta table where there is no HRegionInfo @@ -43,12 +38,14 @@ public class TestEmptyMetaInfo extends HBaseClusterTestCase { */ public void testEmptyMetaInfo() throws IOException { HTable t = new HTable(conf, HConstants.META_TABLE_NAME); - for (int i = 0; i < 5; i++) { - Text regionName = new Text("tablename," + (i == 0 ? "" : (i +",")) + - System.currentTimeMillis()); + final int COUNT = 5; + final byte [] tableName = Bytes.toBytes(getName()); + for (int i = 0; i < COUNT; i++) { + byte [] regionName = HRegionInfo.createRegionName(tableName, + Bytes.toBytes(i == 0? "": Integer.toString(i)), + Long.toString(System.currentTimeMillis())); BatchUpdate b = new BatchUpdate(regionName); - b.put(HConstants.COL_SERVER, - "localhost:1234".getBytes(HConstants.UTF8_ENCODING)); + b.put(HConstants.COL_SERVER, Bytes.toBytes("localhost:1234")); t.commit(b); } long sleepTime = @@ -62,15 +59,13 @@ public class TestEmptyMetaInfo extends HBaseClusterTestCase { } catch (InterruptedException e) { // ignore } - Scanner scanner = - t.getScanner(HConstants.ALL_META_COLUMNS, new Text("tablename")); - + Scanner scanner = t.getScanner(HConstants.ALL_META_COLUMNS, tableName); try { count = 0; - HStoreKey key = new HStoreKey(); - SortedMap results = new TreeMap(); - for (RowResult r : scanner) { - count += 1; + for (RowResult r: scanner) { + if (r.size() > 0) { + count += 1; + } } } finally { scanner.close(); diff --git a/src/test/org/apache/hadoop/hbase/TestGlobalMemcacheLimit.java b/src/test/org/apache/hadoop/hbase/TestGlobalMemcacheLimit.java index c96b4fff92c..6edb863a722 100644 --- a/src/test/org/apache/hadoop/hbase/TestGlobalMemcacheLimit.java +++ b/src/test/org/apache/hadoop/hbase/TestGlobalMemcacheLimit.java @@ -21,13 +21,13 @@ package org.apache.hadoop.hbase; import java.io.IOException; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.util.Bytes; /** * Test setting the global memcache size for a region server. When it reaches @@ -41,7 +41,7 @@ public class TestGlobalMemcacheLimit extends HBaseClusterTestCase { HTable table2; HRegionServer server; - long keySize = (new Text(COLFAMILY_NAME1)).getLength() + 9 + 8; + long keySize = COLFAMILY_NAME1.length + 9 + 8; long rowSize = keySize + ONE_KB.length; /** @@ -69,13 +69,13 @@ public class TestGlobalMemcacheLimit extends HBaseClusterTestCase { HBaseAdmin admin = new HBaseAdmin(conf); admin.createTable(desc1); admin.createTable(desc2); - table1 = new HTable(conf, new Text("testTable1")); - table2 = new HTable(conf, new Text("testTable2")); + table1 = new HTable(conf, "testTable1"); + table2 = new HTable(conf, "testTable2"); server = cluster.getRegionServer(0); // there is a META region in play, and those are probably still in // the memcache for ROOT. flush it out. - for (HRegion region : server.getOnlineRegions().values()) { + for (HRegion region : server.getOnlineRegions()) { region.flushcache(); } // make sure we're starting at 0 so that it's easy to predict what the @@ -122,14 +122,15 @@ public class TestGlobalMemcacheLimit extends HBaseClusterTestCase { assertTrue("Post-flush memcache size", server.getGlobalMemcacheSize() <= 1024 * 1024); } - private long populate(HTable table, int numRows, int startKey) throws IOException { + private long populate(HTable table, int numRows, int startKey) + throws IOException { long total = 0; BatchUpdate batchUpdate = null; - Text column = new Text(COLFAMILY_NAME1); + byte [] column = COLFAMILY_NAME1; for (int i = startKey; i < startKey + numRows; i++) { - Text key = new Text("row_" + String.format("%1$5d", i)); - total += key.getLength(); - total += column.getLength(); + byte [] key = Bytes.toBytes("row_" + String.format("%1$5d", i)); + total += key.length; + total += column.length; total += 8; total += ONE_KB.length; batchUpdate = new BatchUpdate(key); diff --git a/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java b/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java index e87ba1fcdff..e5301f80580 100644 --- a/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java +++ b/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java @@ -20,17 +20,17 @@ package org.apache.hadoop.hbase; import java.io.IOException; +import java.util.Collection; import java.util.Iterator; -import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Scanner; -import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.util.Bytes; /** * Test HBase Master and Region servers, client API @@ -76,17 +76,19 @@ public class TestHBaseCluster extends HBaseClusterTestCase { private static final int FIRST_ROW = 1; private static final int NUM_VALS = 1000; - private static final Text CONTENTS = new Text("contents:"); - private static final Text CONTENTS_BASIC = new Text("contents:basic"); + private static final byte [] CONTENTS = Bytes.toBytes("contents:"); + private static final byte [] CONTENTS_MINUS_COLON = Bytes.toBytes("contents"); + private static final byte [] CONTENTS_BASIC = Bytes.toBytes("contents:basic"); private static final String CONTENTSTR = "contentstr"; - private static final Text ANCHOR = new Text("anchor:"); + private static final byte [] ANCHOR = Bytes.toBytes("anchor:"); + private static final byte [] ANCHOR_MINUS_COLON = Bytes.toBytes("anchor"); private static final String ANCHORNUM = "anchor:anchornum-"; private static final String ANCHORSTR = "anchorstr"; private void setup() throws IOException { desc = new HTableDescriptor("test"); - desc.addFamily(new HColumnDescriptor(CONTENTS.toString())); - desc.addFamily(new HColumnDescriptor(ANCHOR.toString())); + desc.addFamily(new HColumnDescriptor(CONTENTS)); + desc.addFamily(new HColumnDescriptor(ANCHOR)); admin = new HBaseAdmin(conf); admin.createTable(desc); table = new HTable(conf, desc.getName()); @@ -100,11 +102,9 @@ public class TestHBaseCluster extends HBaseClusterTestCase { // Write out a bunch of values for (int k = FIRST_ROW; k <= NUM_VALS; k++) { - BatchUpdate b = new BatchUpdate(new Text("row_" + k)); - b.put(CONTENTS_BASIC, - (CONTENTSTR + k).getBytes(HConstants.UTF8_ENCODING)); - b.put(new Text(ANCHORNUM + k), - (ANCHORSTR + k).getBytes(HConstants.UTF8_ENCODING)); + BatchUpdate b = new BatchUpdate("row_" + k); + b.put(CONTENTS_BASIC, Bytes.toBytes(CONTENTSTR + k)); + b.put(ANCHORNUM + k, Bytes.toBytes(ANCHORSTR + k)); table.commit(b); } LOG.info("Write " + NUM_VALS + " rows. Elapsed time: " @@ -114,9 +114,9 @@ public class TestHBaseCluster extends HBaseClusterTestCase { startTime = System.currentTimeMillis(); - Text collabel = null; + byte [] collabel = null; for (int k = FIRST_ROW; k <= NUM_VALS; k++) { - Text rowlabel = new Text("row_" + k); + byte [] rowlabel = Bytes.toBytes("row_" + k); byte bodydata[] = table.get(rowlabel, CONTENTS_BASIC).getValue(); assertNotNull("no data for row " + rowlabel + "/" + CONTENTS_BASIC, @@ -127,7 +127,7 @@ public class TestHBaseCluster extends HBaseClusterTestCase { CONTENTS_BASIC + "), expected: '" + teststr + "' got: '" + bodystr + "'", teststr.compareTo(bodystr) == 0); - collabel = new Text(ANCHORNUM + k); + collabel = Bytes.toBytes(ANCHORNUM + k); bodydata = table.get(rowlabel, collabel).getValue(); assertNotNull("no data for row " + rowlabel + "/" + collabel, bodydata); bodystr = new String(bodydata, HConstants.UTF8_ENCODING); @@ -142,33 +142,30 @@ public class TestHBaseCluster extends HBaseClusterTestCase { } private void scanner() throws IOException { - Text[] cols = new Text[] { - new Text(ANCHORNUM + "[0-9]+"), - new Text(CONTENTS_BASIC) - }; + byte [][] cols = new byte [][] {Bytes.toBytes(ANCHORNUM + "[0-9]+"), + CONTENTS_BASIC}; long startTime = System.currentTimeMillis(); - Scanner s = table.getScanner(cols, new Text()); + Scanner s = table.getScanner(cols, HConstants.EMPTY_BYTE_ARRAY); try { int contentsFetched = 0; int anchorFetched = 0; int k = 0; for (RowResult curVals : s) { - for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { - Text col = it.next(); + for (Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { + byte [] col = it.next(); byte val[] = curVals.get(col).getValue(); - String curval = new String(val, HConstants.UTF8_ENCODING).trim(); - - if(col.compareTo(CONTENTS_BASIC) == 0) { + String curval = Bytes.toString(val); + if (Bytes.compareTo(col, CONTENTS_BASIC) == 0) { assertTrue("Error at:" + curVals.getRow() + ", Value for " + col + " should start with: " + CONTENTSTR + ", but was fetched as: " + curval, curval.startsWith(CONTENTSTR)); contentsFetched++; - } else if(col.toString().startsWith(ANCHORNUM)) { + } else if (Bytes.toString(col).startsWith(ANCHORNUM)) { assertTrue("Error at:" + curVals.getRow() + ", Value for " + col + " should start with: " + ANCHORSTR + ", but was fetched as: " + curval, @@ -176,13 +173,18 @@ public class TestHBaseCluster extends HBaseClusterTestCase { anchorFetched++; } else { - LOG.info(col); + LOG.info(Bytes.toString(col)); } } k++; } - assertEquals("Expected " + NUM_VALS + " " + CONTENTS_BASIC + " values, but fetched " + contentsFetched, NUM_VALS, contentsFetched); - assertEquals("Expected " + NUM_VALS + " " + ANCHORNUM + " values, but fetched " + anchorFetched, NUM_VALS, anchorFetched); + assertEquals("Expected " + NUM_VALS + " " + + Bytes.toString(CONTENTS_BASIC) + " values, but fetched " + + contentsFetched, + NUM_VALS, contentsFetched); + assertEquals("Expected " + NUM_VALS + " " + ANCHORNUM + + " values, but fetched " + anchorFetched, + NUM_VALS, anchorFetched); LOG.info("Scanned " + NUM_VALS + " rows. Elapsed time: " @@ -196,10 +198,10 @@ public class TestHBaseCluster extends HBaseClusterTestCase { private void listTables() throws IOException { HTableDescriptor[] tables = admin.listTables(); assertEquals(1, tables.length); - assertEquals(desc.getName(), tables[0].getName()); - Set families = tables[0].families().keySet(); + assertTrue(Bytes.equals(desc.getName(), tables[0].getName())); + Collection families = tables[0].getFamilies(); assertEquals(2, families.size()); - assertTrue(families.contains(new Text(CONTENTS))); - assertTrue(families.contains(new Text(ANCHOR))); + assertTrue(tables[0].hasFamily(CONTENTS)); + assertTrue(tables[0].hasFamily(ANCHOR)); } -} +} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/TestInfoServers.java b/src/test/org/apache/hadoop/hbase/TestInfoServers.java index 97c3e327a9c..d8ac27363e7 100644 --- a/src/test/org/apache/hadoop/hbase/TestInfoServers.java +++ b/src/test/org/apache/hadoop/hbase/TestInfoServers.java @@ -25,7 +25,6 @@ import java.net.URL; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.client.HTable; /** @@ -48,8 +47,7 @@ public class TestInfoServers extends HBaseClusterTestCase { */ public void testInfoServersAreUp() throws Exception { // give the cluster time to start up - HTable table = new HTable(conf, new Text(".META.")); - + new HTable(conf, ".META."); int port = cluster.getMaster().getInfoServer().getPort(); assertHasExpectedContent(new URL("http://localhost:" + port + "/index.html"), "Master"); diff --git a/src/test/org/apache/hadoop/hbase/TestMasterAdmin.java b/src/test/org/apache/hadoop/hbase/TestMasterAdmin.java index 57af6176243..63258fbbd8c 100644 --- a/src/test/org/apache/hadoop/hbase/TestMasterAdmin.java +++ b/src/test/org/apache/hadoop/hbase/TestMasterAdmin.java @@ -21,19 +21,19 @@ package org.apache.hadoop.hbase; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.util.Bytes; /** tests administrative functions */ public class TestMasterAdmin extends HBaseClusterTestCase { private final Log LOG = LogFactory.getLog(this.getClass().getName()); - private static final Text COLUMN_NAME = new Text("col1:"); + private static final byte [] COLUMN_NAME = Bytes.toBytes("col1:"); private static HTableDescriptor testDesc; static { testDesc = new HTableDescriptor("testadmin1"); - testDesc.addFamily(new HColumnDescriptor(COLUMN_NAME.toString())); + testDesc.addFamily(new HColumnDescriptor(COLUMN_NAME)); } private HBaseAdmin admin; @@ -61,9 +61,9 @@ public class TestMasterAdmin extends HBaseClusterTestCase { } assertTrue(exception); admin.createTable(testDesc); - LOG.info("Table " + testDesc.getName().toString() + " created"); + LOG.info("Table " + testDesc.getNameAsString() + " created"); admin.disableTable(testDesc.getName()); - LOG.info("Table " + testDesc.getName().toString() + " disabled"); + LOG.info("Table " + testDesc.getNameAsString() + " disabled"); try { @SuppressWarnings("unused") HTable table = new HTable(conf, testDesc.getName()); @@ -78,13 +78,13 @@ public class TestMasterAdmin extends HBaseClusterTestCase { admin.addColumn(testDesc.getName(), new HColumnDescriptor("col2:")); admin.enableTable(testDesc.getName()); try { - admin.deleteColumn(testDesc.getName(), new Text("col2:")); + admin.deleteColumn(testDesc.getName(), Bytes.toBytes("col2:")); } catch(TableNotDisabledException e) { // Expected } admin.disableTable(testDesc.getName()); - admin.deleteColumn(testDesc.getName(), new Text("col2:")); + admin.deleteColumn(testDesc.getName(), Bytes.toBytes("col2:")); admin.deleteTable(testDesc.getName()); } } diff --git a/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java b/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java index 2d02ea06399..bffe45b0051 100644 --- a/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java +++ b/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java @@ -25,7 +25,6 @@ import java.util.ArrayList; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.client.HTable; @@ -36,6 +35,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.LocalHBaseCluster; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.util.Bytes; /** * Test whether region rebalancing works. (HBASE-71) @@ -48,7 +48,7 @@ public class TestRegionRebalancing extends HBaseClusterTestCase { final byte[] FIVE_HUNDRED_KBYTES; - final Text COLUMN_NAME = new Text("col:"); + final byte [] COLUMN_NAME = Bytes.toBytes("col:"); /** constructor */ public TestRegionRebalancing() { @@ -59,7 +59,7 @@ public class TestRegionRebalancing extends HBaseClusterTestCase { } desc = new HTableDescriptor("test"); - desc.addFamily(new HColumnDescriptor(COLUMN_NAME.toString())); + desc.addFamily(new HColumnDescriptor(COLUMN_NAME)); } /** @@ -68,10 +68,10 @@ public class TestRegionRebalancing extends HBaseClusterTestCase { @Override public void preHBaseClusterSetup() throws IOException { // create a 20-region table by writing directly to disk - List startKeys = new ArrayList(); + List startKeys = new ArrayList(); startKeys.add(null); for (int i = 10; i < 29; i++) { - startKeys.add(new Text("row_" + i)); + startKeys.add(Bytes.toBytes("row_" + i)); } startKeys.add(null); LOG.info(startKeys.size() + " start keys generated"); @@ -84,9 +84,9 @@ public class TestRegionRebalancing extends HBaseClusterTestCase { // Now create the root and meta regions and insert the data regions // created above into the meta - HRegion root = HRegion.createHRegion(HRegionInfo.rootRegionInfo, + HRegion root = HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO, testDir, conf); - HRegion meta = HRegion.createHRegion(HRegionInfo.firstMetaRegionInfo, + HRegion meta = HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO, testDir, conf); HRegion.addRegionToMETA(root, meta); @@ -105,7 +105,7 @@ public class TestRegionRebalancing extends HBaseClusterTestCase { * region servers to see if the assignment or regions is pretty balanced. */ public void testRebalancing() throws IOException { - table = new HTable(conf, new Text("test")); + table = new HTable(conf, "test"); assertEquals("Test table should have 20 regions", 20, table.getStartKeys().length); @@ -220,12 +220,12 @@ public class TestRegionRebalancing extends HBaseClusterTestCase { * create a region with the specified start and end key and exactly one row * inside. */ - private HRegion createAregion(Text startKey, Text endKey) + private HRegion createAregion(byte [] startKey, byte [] endKey) throws IOException { HRegion region = createNewHRegion(desc, startKey, endKey); - Text keyToWrite = startKey == null ? new Text("row_000") : startKey; + byte [] keyToWrite = startKey == null ? Bytes.toBytes("row_000") : startKey; BatchUpdate bu = new BatchUpdate(keyToWrite); bu.put(COLUMN_NAME, "test".getBytes()); diff --git a/src/test/org/apache/hadoop/hbase/TestScannerAPI.java b/src/test/org/apache/hadoop/hbase/TestScannerAPI.java index d0e3be36393..af470f8602a 100644 --- a/src/test/org/apache/hadoop/hbase/TestScannerAPI.java +++ b/src/test/org/apache/hadoop/hbase/TestScannerAPI.java @@ -22,30 +22,30 @@ package org.apache.hadoop.hbase; import java.io.IOException; import java.util.Arrays; -import java.util.Iterator; +import java.util.Collection; import java.util.Map; import java.util.SortedMap; import java.util.TreeMap; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Scanner; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.Text; /** test the scanner API at all levels */ public class TestScannerAPI extends HBaseClusterTestCase { - private final Text[] columns = new Text[] { + private final byte [][] columns = Bytes.toByteArrays(new Text[] { new Text("a:"), new Text("b:") - }; - private final Text startRow = new Text("0"); + }); + private final byte [] startRow = Bytes.toBytes("0"); - private final TreeMap> values = - new TreeMap>(); + private final TreeMap> values = + new TreeMap>(Bytes.BYTES_COMPARATOR); /** * @throws Exception @@ -53,12 +53,13 @@ public class TestScannerAPI extends HBaseClusterTestCase { public TestScannerAPI() throws Exception { super(); try { - TreeMap columns = new TreeMap(); - columns.put(new Text("a:1"), "1".getBytes(HConstants.UTF8_ENCODING)); - values.put(new Text("1"), columns); - columns = new TreeMap(); - columns.put(new Text("a:2"), "2".getBytes(HConstants.UTF8_ENCODING)); - columns.put(new Text("b:2"), "2".getBytes(HConstants.UTF8_ENCODING)); + TreeMap columns = + new TreeMap(Bytes.BYTES_COMPARATOR); + columns.put(Bytes.toBytes("a:1"), Bytes.toBytes("1")); + values.put(Bytes.toBytes("1"), columns); + columns = new TreeMap(Bytes.BYTES_COMPARATOR); + columns.put(Bytes.toBytes("a:2"), Bytes.toBytes("2")); + columns.put(Bytes.toBytes("b:2"), Bytes.toBytes("2")); } catch (Exception e) { e.printStackTrace(); throw e; @@ -76,17 +77,17 @@ public class TestScannerAPI extends HBaseClusterTestCase { HBaseAdmin admin = new HBaseAdmin(conf); HTableDescriptor tableDesc = new HTableDescriptor(tableName); for (int i = 0; i < columns.length; i++) { - tableDesc.addFamily(new HColumnDescriptor(columns[i].toString())); + tableDesc.addFamily(new HColumnDescriptor(columns[i])); } admin.createTable(tableDesc); // Insert values - HTable table = new HTable(conf, new Text(getName())); + HTable table = new HTable(conf, getName()); - for (Map.Entry> row: values.entrySet()) { + for (Map.Entry> row: values.entrySet()) { BatchUpdate b = new BatchUpdate(row.getKey()); - for (Map.Entry val: row.getValue().entrySet()) { + for (Map.Entry val: row.getValue().entrySet()) { b.put(val.getKey(), val.getValue()); } table.commit(b); @@ -94,11 +95,11 @@ public class TestScannerAPI extends HBaseClusterTestCase { HRegion region = null; try { - Map regions = + Collection regions = cluster.getRegionThreads().get(0).getRegionServer().getOnlineRegions(); - for (Map.Entry e: regions.entrySet()) { - if (!e.getValue().getRegionInfo().isMetaRegion()) { - region = e.getValue(); + for (HRegion r: regions) { + if (!r.getRegionInfo().isMetaRegion()) { + region = r; } } } catch (Exception e) { @@ -127,10 +128,10 @@ public class TestScannerAPI extends HBaseClusterTestCase { for (RowResult r : scanner2) { assertTrue("row key", values.containsKey(r.getRow())); - SortedMap columnValues = values.get(r.getRow()); - assertEquals(columnValues.size(), r.size()); - for (Map.Entry e: columnValues.entrySet()) { - Text column = e.getKey(); + SortedMap columnValues = values.get(r.getRow()); + assertEquals(columnValues.size(), r.size()); + for (Map.Entry e: columnValues.entrySet()) { + byte [] column = e.getKey(); assertTrue("column", r.containsKey(column)); assertTrue("value", Arrays.equals(columnValues.get(column), r.get(column).getValue())); @@ -143,15 +144,16 @@ public class TestScannerAPI extends HBaseClusterTestCase { private void verify(ScannerIncommon scanner) throws IOException { HStoreKey key = new HStoreKey(); - SortedMap results = new TreeMap(); + SortedMap results = + new TreeMap(Bytes.BYTES_COMPARATOR); while (scanner.next(key, results)) { - Text row = key.getRow(); + byte [] row = key.getRow(); assertTrue("row key", values.containsKey(row)); - SortedMap columnValues = values.get(row); + SortedMap columnValues = values.get(row); assertEquals(columnValues.size(), results.size()); - for (Map.Entry e: columnValues.entrySet()) { - Text column = e.getKey(); + for (Map.Entry e: columnValues.entrySet()) { + byte [] column = e.getKey(); assertTrue("column", results.containsKey(column)); assertTrue("value", Arrays.equals(columnValues.get(column), results.get(column))); @@ -159,4 +161,4 @@ public class TestScannerAPI extends HBaseClusterTestCase { results.clear(); } } -} +} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/TestSerialization.java b/src/test/org/apache/hadoop/hbase/TestSerialization.java index 1fe7416334a..78044c9bf80 100644 --- a/src/test/org/apache/hadoop/hbase/TestSerialization.java +++ b/src/test/org/apache/hadoop/hbase/TestSerialization.java @@ -19,17 +19,18 @@ */ package org.apache.hadoop.hbase; -import junit.framework.TestCase; - import org.apache.hadoop.hbase.io.BatchOperation; import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.HbaseMapWritable; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.io.Text; /** * Test HBase Writables serializations */ -public class TestSerialization extends TestCase { +public class TestSerialization extends HBaseTestCase { protected void setUp() throws Exception { super.setUp(); @@ -39,6 +40,64 @@ public class TestSerialization extends TestCase { super.tearDown(); } + public void testname() throws Exception { + HMsg m = new HMsg(HMsg.MSG_REGIONSERVER_QUIESCE); + byte [] mb = Writables.getBytes(m); + HMsg deserializedHMsg = (HMsg)Writables.getWritable(mb, new HMsg()); + assertTrue(m.getMsg() == deserializedHMsg.getMsg()); + } + + public void testTableDescriptor() throws Exception { + HTableDescriptor htd = createTableDescriptor(getName()); + byte [] mb = Writables.getBytes(htd); + HTableDescriptor deserializedHtd = + (HTableDescriptor)Writables.getWritable(mb, new HTableDescriptor()); + assertEquals(htd.getNameAsString(), deserializedHtd.getNameAsString()); + } + + /** + * Test RegionInfo serialization + * @throws Exception + */ + public void testRowResult() throws Exception { + HbaseMapWritable m = new HbaseMapWritable(); + byte [] b = Bytes.toBytes(getName()); + m.put(b, new Cell(b, System.currentTimeMillis())); + RowResult rr = new RowResult(b, m); + byte [] mb = Writables.getBytes(rr); + RowResult deserializedRr = + (RowResult)Writables.getWritable(mb, new RowResult()); + assertTrue(Bytes.equals(rr.getRow(), deserializedRr.getRow())); + byte [] one = rr.get(b).getValue(); + byte [] two = deserializedRr.get(b).getValue(); + assertTrue(Bytes.equals(one, two)); + Writables.copyWritable(rr, deserializedRr); + one = rr.get(b).getValue(); + two = deserializedRr.get(b).getValue(); + assertTrue(Bytes.equals(one, two)); + + } + + /** + * Test RegionInfo serialization + * @throws Exception + */ + public void testRegionInfo() throws Exception { + HTableDescriptor htd = new HTableDescriptor(getName()); + String [] families = new String [] {"info:", "anchor:"}; + for (int i = 0; i < families.length; i++) { + htd.addFamily(new HColumnDescriptor(families[i])); + } + HRegionInfo hri = new HRegionInfo(htd, + HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); + byte [] hrib = Writables.getBytes(hri); + HRegionInfo deserializedHri = + (HRegionInfo)Writables.getWritable(hrib, new HRegionInfo()); + assertEquals(hri.getEncodedName(), deserializedHri.getEncodedName()); + assertEquals(hri.getTableDesc().getFamilies().size(), + deserializedHri.getTableDesc().getFamilies().size()); + } + /** * Test ServerInfo serialization * @throws Exception @@ -57,16 +116,15 @@ public class TestSerialization extends TestCase { * @throws Exception */ public void testBatchUpdate() throws Exception { - final Text testName = new Text(getName()); // Add row named 'testName'. - BatchUpdate bu = new BatchUpdate(testName); + BatchUpdate bu = new BatchUpdate(getName()); // Add a column named same as row. - bu.put(testName, testName.getBytes()); + bu.put(getName(), getName().getBytes()); byte [] b = Writables.getBytes(bu); BatchUpdate bubu = (BatchUpdate)Writables.getWritable(b, new BatchUpdate()); // Assert rows are same. - assertTrue(bu.getRow().equals(bubu.getRow())); + assertTrue(Bytes.equals(bu.getRow(), bubu.getRow())); // Assert has same number of BatchOperations. int firstCount = 0; for (BatchOperation bo: bubu) { @@ -76,7 +134,7 @@ public class TestSerialization extends TestCase { // accumulating BatchOperations on each deserialization. BatchUpdate bububu = (BatchUpdate)Writables.getWritable(b, bubu); // Assert rows are same again. - assertTrue(bu.getRow().equals(bububu.getRow())); + assertTrue(Bytes.equals(bu.getRow(), bububu.getRow())); int secondCount = 0; for (BatchOperation bo: bububu) { secondCount++; diff --git a/src/test/org/apache/hadoop/hbase/TestTable.java b/src/test/org/apache/hadoop/hbase/TestTable.java index 29b3aed3ffc..11c608091d7 100644 --- a/src/test/org/apache/hadoop/hbase/TestTable.java +++ b/src/test/org/apache/hadoop/hbase/TestTable.java @@ -35,28 +35,28 @@ public class TestTable extends HBaseClusterTestCase { final HBaseAdmin admin = new HBaseAdmin(conf); String msg = null; try { - admin.createTable(HTableDescriptor.rootTableDesc); + admin.createTable(HTableDescriptor.ROOT_TABLEDESC); } catch (IllegalArgumentException e) { msg = e.toString(); } assertTrue("Unexcepted exception message " + msg, msg != null && msg.startsWith(IllegalArgumentException.class.getName()) && - msg.contains(HTableDescriptor.rootTableDesc.getName().toString())); + msg.contains(HTableDescriptor.ROOT_TABLEDESC.getNameAsString())); msg = null; try { - admin.createTable(HTableDescriptor.metaTableDesc); + admin.createTable(HTableDescriptor.META_TABLEDESC); } catch(IllegalArgumentException e) { msg = e.toString(); } assertTrue("Unexcepted exception message " + msg, msg != null && msg.startsWith(IllegalArgumentException.class.getName()) && - msg.contains(HTableDescriptor.metaTableDesc.getName().toString())); + msg.contains(HTableDescriptor.META_TABLEDESC.getNameAsString())); // Try doing a duplicate database create. msg = null; HTableDescriptor desc = new HTableDescriptor(getName()); - desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString())); + desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY)); admin.createTable(desc); assertTrue("First table creation completed", admin.listTables().length == 1); boolean gotException = false; @@ -72,9 +72,8 @@ public class TestTable extends HBaseClusterTestCase { // Now try and do concurrent creation with a bunch of threads. final HTableDescriptor threadDesc = - new HTableDescriptor("threaded-" + getName()); - threadDesc.addFamily(new HColumnDescriptor(HConstants. - COLUMN_FAMILY.toString())); + new HTableDescriptor("threaded_" + getName()); + threadDesc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY)); int count = 10; Thread [] threads = new Thread [count]; final AtomicInteger successes = new AtomicInteger(0); @@ -123,6 +122,6 @@ public class TestTable extends HBaseClusterTestCase { admin.createTable(new HTableDescriptor(getName())); // Before fix, below would fail throwing a NoServerForRegionException. @SuppressWarnings("unused") - HTable table = new HTable(conf, new Text(getName())); + HTable table = new HTable(conf, getName()); } -} +} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/TestToString.java b/src/test/org/apache/hadoop/hbase/TestToString.java index 665751f77e3..06e61886a7d 100644 --- a/src/test/org/apache/hadoop/hbase/TestToString.java +++ b/src/test/org/apache/hadoop/hbase/TestToString.java @@ -41,25 +41,25 @@ public class TestToString extends TestCase { * Test HTableDescriptor.toString(); */ public void testHTableDescriptor() { - HTableDescriptor htd = HTableDescriptor.rootTableDesc; + HTableDescriptor htd = HTableDescriptor.ROOT_TABLEDESC; System. out.println(htd.toString()); - assertEquals("Table descriptor", "name: -ROOT-, families: {info:={name: " + + assertEquals("Table descriptor", "name: -ROOT-, families: [{name: " + "info, max versions: 1, compression: NONE, in memory: false, " + "block cache enabled: false, max length: 2147483647, " + - "time to live: FOREVER, bloom filter: none}}", htd.toString()); + "time to live: FOREVER, bloom filter: none}]", htd.toString()); } /** * Tests HRegionInfo.toString() */ public void testHRegionInfo() { - HRegionInfo hri = HRegionInfo.rootRegionInfo; + HRegionInfo hri = HRegionInfo.ROOT_REGIONINFO; System.out.println(hri.toString()); assertEquals("HRegionInfo", "regionname: -ROOT-,,0, startKey: <>, endKey: <>, encodedName: 70236052, tableDesc: " + - "{name: -ROOT-, families: {info:={name: info, max versions: 1, " + + "{name: -ROOT-, families: [{name: info, max versions: 1, " + "compression: NONE, in memory: false, block cache enabled: false, " + - "max length: 2147483647, time to live: FOREVER, bloom filter: none}}}", + "max length: 2147483647, time to live: FOREVER, bloom filter: none}]}", hri.toString()); } } diff --git a/src/test/org/apache/hadoop/hbase/TimestampTestBase.java b/src/test/org/apache/hadoop/hbase/TimestampTestBase.java index 0b323a4fcdb..1cff1105858 100644 --- a/src/test/org/apache/hadoop/hbase/TimestampTestBase.java +++ b/src/test/org/apache/hadoop/hbase/TimestampTestBase.java @@ -21,22 +21,11 @@ package org.apache.hadoop.hbase; import java.io.IOException; import java.util.TreeMap; -import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType; -import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.Scanner; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HStoreKey; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.HBaseTestCase; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.util.Bytes; /** * Tests user specifiable time stamps putting, getting and scanning. Also @@ -53,12 +42,9 @@ public class TimestampTestBase extends HBaseTestCase { private static final String COLUMN_NAME = "contents:"; - private static final Text COLUMN = new Text(COLUMN_NAME); - private static final Text ROW = new Text("row"); - - // When creating column descriptor, how many versions of a cell to allow. - private static final int VERSIONS = 3; - + private static final byte [] COLUMN = Bytes.toBytes(COLUMN_NAME); + private static final byte [] ROW = Bytes.toBytes("row"); + /* * Run test that delete works according to description in hadoop-1784. @@ -115,7 +101,7 @@ public class TimestampTestBase extends HBaseTestCase { throws IOException { Cell[] cellValues = incommon.get(ROW, COLUMN, 3/*Ask for too much*/); assertEquals(1, cellValues.length); - long time = Writables.bytesToLong(cellValues[0].getValue()); + long time = Bytes.toLong(cellValues[0].getValue()); assertEquals(time, currentTime); assertNull(incommon.get(ROW, COLUMN, T1, 3 /*Too many*/)); assertTrue(assertScanContentTimestamp(incommon, T1) == 0); @@ -133,19 +119,19 @@ public class TimestampTestBase extends HBaseTestCase { throws IOException { // Assert that 'latest' is what we expect. byte [] bytes = incommon.get(ROW, COLUMN).getValue(); - assertEquals(Writables.bytesToLong(bytes), tss[0]); + assertEquals(Bytes.toLong(bytes), tss[0]); // Now assert that if we ask for multiple versions, that they come out in // order. Cell[] cellValues = incommon.get(ROW, COLUMN, tss.length); assertEquals(tss.length, cellValues.length); for (int i = 0; i < cellValues.length; i++) { - long ts = Writables.bytesToLong(cellValues[i].getValue()); + long ts = Bytes.toLong(cellValues[i].getValue()); assertEquals(ts, tss[i]); } // Specify a timestamp get multiple versions. cellValues = incommon.get(ROW, COLUMN, tss[0], cellValues.length - 1); for (int i = 1; i < cellValues.length; i++) { - long ts = Writables.bytesToLong(cellValues[i].getValue()); + long ts = Bytes.toLong(cellValues[i].getValue()); assertEquals(ts, tss[i]); } // Test scanner returns expected version @@ -191,12 +177,13 @@ public class TimestampTestBase extends HBaseTestCase { int count = 0; try { HStoreKey key = new HStoreKey(); - TreeMapvalue = new TreeMap(); + TreeMapvalue = + new TreeMap(Bytes.BYTES_COMPARATOR); while (scanner.next(key, value)) { assertTrue(key.getTimestamp() <= ts); // Content matches the key or HConstants.LATEST_TIMESTAMP. // (Key does not match content if we 'put' with LATEST_TIMESTAMP). - long l = Writables.bytesToLong(value.get(COLUMN)); + long l = Bytes.toLong(value.get(COLUMN)); assertTrue(key.getTimestamp() == l || HConstants.LATEST_TIMESTAMP == l); count++; @@ -210,13 +197,13 @@ public class TimestampTestBase extends HBaseTestCase { public static void put(final Incommon loader, final long ts) throws IOException { - put(loader, Writables.longToBytes(ts), ts); + put(loader, Bytes.toBytes(ts), ts); } public static void put(final Incommon loader) throws IOException { long ts = HConstants.LATEST_TIMESTAMP; - put(loader, Writables.longToBytes(ts), ts); + put(loader, Bytes.toBytes(ts), ts); } /* diff --git a/src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java b/src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java index 0b8dc668d5e..ba6748daf7a 100644 --- a/src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java +++ b/src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java @@ -30,14 +30,14 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.util.Bytes; /** * Test batch updates */ public class TestBatchUpdate extends HBaseClusterTestCase { private static final String CONTENTS_STR = "contents:"; - private static final Text CONTENTS = new Text(CONTENTS_STR); + private static final byte [] CONTENTS = Bytes.toBytes(CONTENTS_STR); private byte[] value; private HTableDescriptor desc = null; @@ -68,19 +68,19 @@ public class TestBatchUpdate extends HBaseClusterTestCase { * @throws IOException */ public void testBatchUpdate() throws IOException { - BatchUpdate bu = new BatchUpdate(new Text("row1")); + BatchUpdate bu = new BatchUpdate("row1"); bu.put(CONTENTS, value); bu.delete(CONTENTS); table.commit(bu); - bu = new BatchUpdate(new Text("row2")); + bu = new BatchUpdate("row2"); bu.put(CONTENTS, value); table.commit(bu); - Text[] columns = { CONTENTS }; - Scanner scanner = table.getScanner(columns, new Text()); + byte [][] columns = { CONTENTS }; + Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW); for (RowResult r : scanner) { - for(Map.Entry e: r.entrySet()) { + for(Map.Entry e: r.entrySet()) { System.out.println(r.getRow() + ": row: " + e.getKey() + " value: " + new String(e.getValue().getValue(), HConstants.UTF8_ENCODING)); } diff --git a/src/test/org/apache/hadoop/hbase/client/TestHTable.java b/src/test/org/apache/hadoop/hbase/client/TestHTable.java index 01ab1634b2a..6060fa46e88 100644 --- a/src/test/org/apache/hadoop/hbase/client/TestHTable.java +++ b/src/test/org/apache/hadoop/hbase/client/TestHTable.java @@ -21,21 +21,19 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; import java.util.Map; -import java.util.TreeMap; - -import org.apache.hadoop.io.Text; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HBaseClusterTestCase; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HStoreKey; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.Text; /** * Tests HTable @@ -43,13 +41,13 @@ import org.apache.hadoop.hbase.io.BatchUpdate; public class TestHTable extends HBaseClusterTestCase implements HConstants { private static final Log LOG = LogFactory.getLog(TestHTable.class); private static final HColumnDescriptor column = - new HColumnDescriptor(COLUMN_FAMILY.toString()); + new HColumnDescriptor(COLUMN_FAMILY); - private static final Text nosuchTable = new Text("nosuchTable"); - private static final Text tableAname = new Text("tableA"); - private static final Text tableBname = new Text("tableB"); + private static final byte [] nosuchTable = Bytes.toBytes("nosuchTable"); + private static final byte [] tableAname = Bytes.toBytes("tableA"); + private static final byte [] tableBname = Bytes.toBytes("tableB"); - private static final Text row = new Text("row"); + private static final byte [] row = Bytes.toBytes("row"); /** * the test @@ -69,10 +67,10 @@ public class TestHTable extends HBaseClusterTestCase implements HConstants { fail(); } - HTableDescriptor tableAdesc = new HTableDescriptor(tableAname.toString()); + HTableDescriptor tableAdesc = new HTableDescriptor(tableAname); tableAdesc.addFamily(column); - HTableDescriptor tableBdesc = new HTableDescriptor(tableBname.toString()); + HTableDescriptor tableBdesc = new HTableDescriptor(tableBname); tableBdesc.addFamily(column); // create a couple of tables @@ -106,7 +104,7 @@ public class TestHTable extends HBaseClusterTestCase implements HConstants { try { for (RowResult r : s) { batchUpdate = new BatchUpdate(r.getRow()); - for(Map.Entry e: r.entrySet()) { + for(Map.Entry e: r.entrySet()) { batchUpdate.put(e.getKey(), e.getValue().getValue()); } b.commit(batchUpdate); @@ -153,7 +151,7 @@ public class TestHTable extends HBaseClusterTestCase implements HConstants { public void testTableNotFoundExceptionWithATable() { try { HColumnDescriptor column = - new HColumnDescriptor(COLUMN_FAMILY.toString()); + new HColumnDescriptor(COLUMN_FAMILY); HBaseAdmin admin = new HBaseAdmin(conf); HTableDescriptor testTableADesc = new HTableDescriptor("table"); diff --git a/src/test/org/apache/hadoop/hbase/client/TestListTables.java b/src/test/org/apache/hadoop/hbase/client/TestListTables.java index afd984d60e6..def96dd7e47 100644 --- a/src/test/org/apache/hadoop/hbase/client/TestListTables.java +++ b/src/test/org/apache/hadoop/hbase/client/TestListTables.java @@ -20,12 +20,12 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; -import java.util.Arrays; import java.util.HashSet; + import org.apache.hadoop.hbase.HBaseClusterTestCase; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; /** * Tests the listTables client API @@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.HConstants; public class TestListTables extends HBaseClusterTestCase { HBaseAdmin admin = null; - private static final HTableDescriptor[] tables = { + private static final HTableDescriptor[] TABLES = { new HTableDescriptor("table1"), new HTableDescriptor("table2"), new HTableDescriptor("table3") @@ -43,15 +43,12 @@ public class TestListTables extends HBaseClusterTestCase { @Override public void setUp() throws Exception { super.setUp(); - admin = new HBaseAdmin(conf); - HColumnDescriptor family = new HColumnDescriptor(HConstants.COLUMN_FAMILY_STR); - - for (int i = 0; i < tables.length; i++) { - tables[i].addFamily(family); - admin.createTable(tables[i]); + for (int i = 0; i < TABLES.length; i++) { + TABLES[i].addFamily(family); + admin.createTable(TABLES[i]); } } @@ -60,13 +57,15 @@ public class TestListTables extends HBaseClusterTestCase { * @throws IOException */ public void testListTables() throws IOException { - HashSet result = - new HashSet(Arrays.asList(admin.listTables())); - + HTableDescriptor [] ts = admin.listTables(); + HashSet result = new HashSet(ts.length); + for (int i = 0; i < ts.length; i++) { + result.add(ts[i]); + } int size = result.size(); - assertEquals(tables.length, size); - for (int i = 0; i < tables.length && i < size; i++) { - assertTrue(result.contains(tables[i])); + assertEquals(TABLES.length, size); + for (int i = 0; i < TABLES.length && i < size; i++) { + assertTrue(result.contains(TABLES[i])); } } -} +} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/client/TestTimestamp.java b/src/test/org/apache/hadoop/hbase/client/TestTimestamp.java index f39efac7a74..f9286db76fc 100644 --- a/src/test/org/apache/hadoop/hbase/client/TestTimestamp.java +++ b/src/test/org/apache/hadoop/hbase/client/TestTimestamp.java @@ -19,24 +19,14 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; -import java.util.TreeMap; - -import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType; -import org.apache.hadoop.hbase.util.Writables; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.Scanner; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HStoreKey; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.hbase.HBaseClusterTestCase; -import org.apache.hadoop.hbase.TimestampTestBase; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseClusterTestCase; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TimestampTestBase; +import org.apache.hadoop.hbase.util.Bytes; /** * Tests user specifiable time stamps putting, getting and scanning. Also @@ -48,7 +38,7 @@ public class TestTimestamp extends HBaseClusterTestCase { LogFactory.getLog(TestTimestamp.class.getName()); private static final String COLUMN_NAME = "contents:"; - private static final Text COLUMN = new Text(COLUMN_NAME); + private static final byte [] COLUMN = Bytes.toBytes(COLUMN_NAME); // When creating column descriptor, how many versions of a cell to allow. private static final int VERSIONS = 3; @@ -90,6 +80,6 @@ public class TestTimestamp extends HBaseClusterTestCase { desc.addFamily(new HColumnDescriptor(COLUMN_NAME)); HBaseAdmin admin = new HBaseAdmin(conf); admin.createTable(desc); - return new HTable(conf, new Text(getName())); + return new HTable(conf, getName()); } } diff --git a/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java b/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java index c7d752cfdca..76a0de59477 100644 --- a/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java +++ b/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java @@ -24,7 +24,7 @@ import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.util.Bytes; import junit.framework.TestCase; @@ -32,9 +32,9 @@ import junit.framework.TestCase; * Tests the inclusive stop row filter */ public class TestInclusiveStopRowFilter extends TestCase { - private final Text STOP_ROW = new Text("stop_row"); - private final Text GOOD_ROW = new Text("good_row"); - private final Text PAST_STOP_ROW = new Text("zzzzzz"); + private final byte [] STOP_ROW = Bytes.toBytes("stop_row"); + private final byte [] GOOD_ROW = Bytes.toBytes("good_row"); + private final byte [] PAST_STOP_ROW = Bytes.toBytes("zzzzzz"); RowFilterInterface mainFilter; diff --git a/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java b/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java index ab1066c50cf..4bc82cb5c76 100644 --- a/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java +++ b/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java @@ -24,7 +24,8 @@ import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.util.Bytes; + import junit.framework.TestCase; @@ -82,7 +83,7 @@ public class TestPageRowFilter extends TestCase { private void testFiltersBeyondPageSize(final RowFilterInterface filter, final int pageSize) { for (int i = 0; i < (pageSize * 2); i++) { - Text row = new Text(Integer.toString(i)); + byte [] row = Bytes.toBytes(Integer.toString(i)); boolean filterOut = filter.filterRowKey(row); if (!filterOut) { assertFalse("Disagrees with 'filter'", filter.filterAllRemaining()); diff --git a/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java b/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java index 542fa71f1ff..79f4c61e7c6 100644 --- a/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java +++ b/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java @@ -31,13 +31,13 @@ import junit.framework.TestCase; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.regionserver.HLogEdit; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.util.Bytes; /** * Tests for regular expression row filter */ public class TestRegExpRowFilter extends TestCase { - TreeMap colvalues; + TreeMap colvalues; RowFilterInterface mainFilter; final char FIRST_CHAR = 'a'; final char LAST_CHAR = 'e'; @@ -55,9 +55,9 @@ public class TestRegExpRowFilter extends TestCase { @Override protected void setUp() throws Exception { super.setUp(); - this.colvalues = new TreeMap(); + this.colvalues = new TreeMap(Bytes.BYTES_COMPARATOR); for (char c = FIRST_CHAR; c < LAST_CHAR; c++) { - colvalues.put(new Text(new String(new char [] {c})), GOOD_BYTES); + colvalues.put(Bytes.toBytes(new String(new char [] {c})), GOOD_BYTES); } this.mainFilter = new RegExpRowFilter(HOST_PREFIX + ".*", colvalues); } @@ -113,28 +113,28 @@ public class TestRegExpRowFilter extends TestCase { private void regexRowTests(RowFilterInterface filter) throws Exception { for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) { - Text t = createRow(c); + byte [] t = createRow(c); assertFalse("Failed with characer " + c, filter.filterRowKey(t)); } String yahooSite = "com.yahoo.www"; assertTrue("Failed with character " + - yahooSite, filter.filterRowKey(new Text(yahooSite))); + yahooSite, filter.filterRowKey(Bytes.toBytes(yahooSite))); } private void regexRowColumnTests(RowFilterInterface filter) throws UnsupportedEncodingException { for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) { - Text t = createRow(c); - for (Map.Entry e: this.colvalues.entrySet()) { + byte [] t = createRow(c); + for (Map.Entry e: this.colvalues.entrySet()) { assertFalse("Failed on " + c, filter.filterColumn(t, e.getKey(), e.getValue())); } } // Try a row and column I know will pass. char c = 'c'; - Text r = createRow(c); - Text col = new Text(Character.toString(c)); + byte [] r = createRow(c); + byte [] col = Bytes.toBytes(Character.toString(c)); assertFalse("Failed with character " + c, filter.filterColumn(r, col, GOOD_BYTES)); @@ -144,24 +144,24 @@ public class TestRegExpRowFilter extends TestCase { // Do with good bytes but bad column name. Should not filter out. assertFalse("Failed with character " + c, - filter.filterColumn(r, new Text("badcolumn"), GOOD_BYTES)); + filter.filterColumn(r, Bytes.toBytes("badcolumn"), GOOD_BYTES)); // Good column, good bytes but bad row. assertTrue("Failed with character " + c, - filter.filterColumn(new Text("bad row"), new Text("badcolumn"), GOOD_BYTES)); + filter.filterColumn(Bytes.toBytes("bad row"), + Bytes.toBytes("badcolumn"), GOOD_BYTES)); } private void filterNotNullTests(RowFilterInterface filter) throws Exception { // Modify the filter to expect certain columns to be null: // Expecting a row WITH columnKeys: a-d, WITHOUT columnKey: e - ((RegExpRowFilter)filter).setColumnFilter(new Text(new String(new char[] { - LAST_CHAR })), null); + ((RegExpRowFilter)filter).setColumnFilter(new byte [] {LAST_CHAR}, null); char secondToLast = (char)(LAST_CHAR - 1); char thirdToLast = (char)(LAST_CHAR - 2); // Modify the row to be missing an expected columnKey (d) - colvalues.remove(new Text(new String(new char[] { secondToLast }))); + colvalues.remove(new byte [] {(byte)secondToLast}); // Try a row that is missing an expected columnKey. // Testing row with columnKeys: a-c @@ -171,27 +171,26 @@ public class TestRegExpRowFilter extends TestCase { // Try a row that has all expected columnKeys, and NO null-expected // columnKeys. // Testing row with columnKeys: a-d - colvalues.put(new Text(new String(new char[] { secondToLast })), - GOOD_BYTES); + colvalues.put(new byte [] {(byte)secondToLast}, GOOD_BYTES); assertFalse("Failed with last columnKey " + secondToLast, filter. filterRow(colvalues)); // Try a row that has all expected columnKeys AND a null-expected columnKey. // Testing row with columnKeys: a-e - colvalues.put(new Text(new String(new char[] { LAST_CHAR })), GOOD_BYTES); + colvalues.put(new byte [] {LAST_CHAR}, GOOD_BYTES); assertTrue("Failed with last columnKey " + LAST_CHAR, filter. filterRow(colvalues)); // Try a row that has all expected columnKeys and a null-expected columnKey // 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 })), + colvalues.put(new byte [] {LAST_CHAR}, HLogEdit.deleteBytes.get()); assertFalse("Failed with last columnKey " + LAST_CHAR + " mapping to null.", filter.filterRow(colvalues)); } - private Text createRow(final char c) { - return new Text(HOST_PREFIX + Character.toString(c)); + private byte [] createRow(final char c) { + return Bytes.toBytes(HOST_PREFIX + Character.toString(c)); } } diff --git a/src/test/org/apache/hadoop/hbase/filter/TestRowFilterAfterWrite.java b/src/test/org/apache/hadoop/hbase/filter/TestRowFilterAfterWrite.java index 07e5a334d70..68f81190c04 100644 --- a/src/test/org/apache/hadoop/hbase/filter/TestRowFilterAfterWrite.java +++ b/src/test/org/apache/hadoop/hbase/filter/TestRowFilterAfterWrite.java @@ -24,7 +24,6 @@ import java.io.IOException; import java.util.Arrays; import java.util.HashMap; import java.util.Map; -import java.util.TreeMap; import junit.framework.Assert; @@ -33,15 +32,14 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HBaseClusterTestCase; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HStoreKey; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Scanner; import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.util.Bytes; /** Test regexp filters HBASE-476 */ public class TestRowFilterAfterWrite extends HBaseClusterTestCase { @@ -52,11 +50,11 @@ public class TestRowFilterAfterWrite extends HBaseClusterTestCase { static final String TABLE_NAME = "TestTable"; static final String FAMILY = "C:"; static final String COLUMN1 = FAMILY + "col1"; - static final Text TEXT_COLUMN1 = new Text(COLUMN1); + static final byte [] TEXT_COLUMN1 = Bytes.toBytes(COLUMN1); static final String COLUMN2 = FAMILY + "col2"; - static final Text TEXT_COLUMN2 = new Text(COLUMN2); + static final byte [] TEXT_COLUMN2 = Bytes.toBytes(COLUMN2); - private static final Text[] columns = { + private static final byte [][] columns = { TEXT_COLUMN1, TEXT_COLUMN2 }; @@ -120,11 +118,10 @@ public class TestRowFilterAfterWrite extends HBaseClusterTestCase { admin.createTable(desc); // insert some data into the test table - HTable table = new HTable(conf, new Text(TABLE_NAME)); + HTable table = new HTable(conf, TABLE_NAME); for (int i = 0; i < NUM_ROWS; i++) { - BatchUpdate b = - new BatchUpdate(new Text("row_" + String.format("%1$05d", i))); + BatchUpdate b = new BatchUpdate("row_" + String.format("%1$05d", i)); b.put(TEXT_COLUMN1, VALUE); b.put(TEXT_COLUMN2, String.format("%1$05d", i).getBytes()); @@ -139,8 +136,7 @@ public class TestRowFilterAfterWrite extends HBaseClusterTestCase { // Do some identity write operations on one column of the data. for (int n = 0; n < NUM_REWRITES; n++) { for (int i = 0; i < NUM_ROWS; i++) { - BatchUpdate b = - new BatchUpdate(new Text("row_" + String.format("%1$05d", i))); + BatchUpdate b = new BatchUpdate("row_" + String.format("%1$05d", i)); b.put(TEXT_COLUMN2, String.format("%1$05d", i).getBytes()); table.commit(b); @@ -155,9 +151,7 @@ public class TestRowFilterAfterWrite extends HBaseClusterTestCase { // Do another round so to populate the mem cache for (int i = 0; i < NUM_ROWS; i++) { - BatchUpdate b = - new BatchUpdate(new Text("row_" + String.format("%1$05d", i))); - + BatchUpdate b = new BatchUpdate("row_" + String.format("%1$05d", i)); b.put(TEXT_COLUMN2, String.format("%1$05d", i).getBytes()); table.commit(b); } @@ -169,7 +163,7 @@ public class TestRowFilterAfterWrite extends HBaseClusterTestCase { } private void scanTable(final String tableName, final boolean printValues) throws IOException { - HTable table = new HTable(conf, new Text(tableName)); + HTable table = new HTable(conf, tableName); Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW); int numFound = doScan(scanner, printValues); @@ -177,8 +171,8 @@ public class TestRowFilterAfterWrite extends HBaseClusterTestCase { } private void scanTableWithRowFilter(final String tableName, final boolean printValues) throws IOException { - HTable table = new HTable(conf, new Text(tableName)); - Map columnMap = new HashMap(); + HTable table = new HTable(conf, tableName); + Map columnMap = new HashMap(); columnMap.put(TEXT_COLUMN1, VALUE); RegExpRowFilter filter = new RegExpRowFilter(null, columnMap); Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW, filter); @@ -194,8 +188,7 @@ public class TestRowFilterAfterWrite extends HBaseClusterTestCase { for (RowResult result : scanner) { if (printValues) { LOG.info("row: " + result.getRow()); - - for (Map.Entry e : result.entrySet()) { + for (Map.Entry e : result.entrySet()) { LOG.info(" column: " + e.getKey() + " value: " + new String(e.getValue().getValue(), HConstants.UTF8_ENCODING)); } diff --git a/src/test/org/apache/hadoop/hbase/filter/TestRowFilterOnMultipleFamilies.java b/src/test/org/apache/hadoop/hbase/filter/TestRowFilterOnMultipleFamilies.java index f6300cf64ab..276f501c5c1 100644 --- a/src/test/org/apache/hadoop/hbase/filter/TestRowFilterOnMultipleFamilies.java +++ b/src/test/org/apache/hadoop/hbase/filter/TestRowFilterOnMultipleFamilies.java @@ -23,7 +23,6 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; import java.util.HashMap; import java.util.Map; -import java.util.TreeMap; import junit.framework.Assert; @@ -32,15 +31,14 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HBaseClusterTestCase; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HStoreKey; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Scanner; import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.io.Cell; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.util.Bytes; /** * Test for regexp filters (HBASE-527) @@ -50,13 +48,11 @@ public class TestRowFilterOnMultipleFamilies extends HBaseClusterTestCase { static final String TABLE_NAME = "TestTable"; static final String COLUMN1 = "A:col1"; - static final Text TEXT_COLUMN1 = new Text(COLUMN1); + static final byte [] TEXT_COLUMN1 = Bytes.toBytes(COLUMN1); static final String COLUMN2 = "B:col2"; - static final Text TEXT_COLUMN2 = new Text(COLUMN2); + static final byte [] TEXT_COLUMN2 = Bytes.toBytes(COLUMN2); - private static final Text[] columns = { - TEXT_COLUMN1, TEXT_COLUMN2 - }; + private static final byte [][] columns = {TEXT_COLUMN1, TEXT_COLUMN2}; private static final int NUM_ROWS = 10; private static final byte[] VALUE = "HELLO".getBytes(); @@ -72,11 +68,10 @@ public class TestRowFilterOnMultipleFamilies extends HBaseClusterTestCase { admin.createTable(desc); // insert some data into the test table - HTable table = new HTable(conf, new Text(TABLE_NAME)); + HTable table = new HTable(conf, TABLE_NAME); for (int i = 0; i < NUM_ROWS; i++) { - BatchUpdate b = - new BatchUpdate(new Text("row_" + String.format("%1$05d", i))); + BatchUpdate b = new BatchUpdate("row_" + String.format("%1$05d", i)); b.put(TEXT_COLUMN1, VALUE); b.put(TEXT_COLUMN2, String.format("%1$05d", i).getBytes()); table.commit(b); @@ -89,16 +84,15 @@ public class TestRowFilterOnMultipleFamilies extends HBaseClusterTestCase { } private void scanTable(final String tableName, final boolean printValues) throws IOException { - HTable table = new HTable(conf, new Text(tableName)); - + HTable table = new HTable(conf, tableName); Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW); int numFound = doScan(scanner, printValues); Assert.assertEquals(NUM_ROWS, numFound); } private void scanTableWithRowFilter(final String tableName, final boolean printValues) throws IOException { - HTable table = new HTable(conf, new Text(tableName)); - Map columnMap = new HashMap(); + HTable table = new HTable(conf, tableName); + Map columnMap = new HashMap(); columnMap.put(TEXT_COLUMN1, VALUE); RegExpRowFilter filter = new RegExpRowFilter(null, columnMap); Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW, filter); @@ -115,7 +109,7 @@ public class TestRowFilterOnMultipleFamilies extends HBaseClusterTestCase { if (printValues) { LOG.info("row: " + result.getRow()); - for (Map.Entry e : result.entrySet()) { + for (Map.Entry e : result.entrySet()) { LOG.info(" column: " + e.getKey() + " value: " + new String(e.getValue().getValue(), HConstants.UTF8_ENCODING)); } diff --git a/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java b/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java index 35e231487d7..8e0eec31294 100644 --- a/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java +++ b/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java @@ -29,7 +29,7 @@ import java.util.Set; import java.util.TreeMap; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.util.Bytes; import junit.framework.TestCase; @@ -44,7 +44,7 @@ public class TestRowFilterSet extends TestCase { static final int MAX_PAGES = 5; final char FIRST_CHAR = 'a'; final char LAST_CHAR = 'e'; - TreeMap colvalues; + TreeMap colvalues; static byte[] GOOD_BYTES = null; static byte[] BAD_BYTES = null; @@ -62,15 +62,15 @@ public class TestRowFilterSet extends TestCase { protected void setUp() throws Exception { super.setUp(); - colvalues = new TreeMap(); + colvalues = new TreeMap(Bytes.BYTES_COMPARATOR); for (char c = FIRST_CHAR; c < LAST_CHAR; c++) { - colvalues.put(new Text(new String(new char [] {c})), GOOD_BYTES); + colvalues.put(new byte [] {(byte)c}, GOOD_BYTES); } Set filters = new HashSet(); filters.add(new PageRowFilter(MAX_PAGES)); filters.add(new RegExpRowFilter(".*regex.*", colvalues)); - filters.add(new WhileMatchRowFilter(new StopRowFilter(new Text("yyy")))); + filters.add(new WhileMatchRowFilter(new StopRowFilter(Bytes.toBytes("yyy")))); filters.add(new WhileMatchRowFilter(new RegExpRowFilter(".*match.*"))); filterMPALL = new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ALL, filters); @@ -132,7 +132,7 @@ public class TestRowFilterSet extends TestCase { // Accept several more rows such that PageRowFilter will exceed its limit. for (int i=0; i<=MAX_PAGES-3; i++) - filter.rowProcessed(false, new Text("unimportant_key")); + filter.rowProcessed(false, Bytes.toBytes("unimportant_key")); // A row that should cause the RegExpRowFilter to filter this row, making // all the filters return true and thus the RowFilterSet as well. @@ -163,15 +163,15 @@ public class TestRowFilterSet extends TestCase { RFSAssertReset(filter); // A row that should cause the RegExpRowFilter to fail. - boolean filtered = filter.filterColumn(new Text("regex_match"), - new Text(new String(new char[] { FIRST_CHAR })), BAD_BYTES); + boolean filtered = filter.filterColumn(Bytes.toBytes("regex_match"), + new byte [] { FIRST_CHAR }, BAD_BYTES); assertTrue("Filtering on 'regex_match' and bad column data.", filtered); - filterMPALL.rowProcessed(filtered, new Text("regex_match")); + filterMPALL.rowProcessed(filtered, Bytes.toBytes("regex_match")); } private void RFSAssertion(RowFilterInterface filter, String toTest, boolean assertTrue) throws Exception { - Text testText = new Text(toTest); + byte [] testText = Bytes.toBytes(toTest); boolean filtered = filter.filterRowKey(testText); assertTrue("Filtering on '" + toTest + "'", assertTrue? filtered : !filtered); diff --git a/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java b/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java index 624713351a5..a2673bb894a 100644 --- a/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java +++ b/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java @@ -24,7 +24,7 @@ import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.util.Bytes; import junit.framework.TestCase; @@ -32,9 +32,9 @@ import junit.framework.TestCase; * Tests the stop row filter */ public class TestStopRowFilter extends TestCase { - private final Text STOP_ROW = new Text("stop_row"); - private final Text GOOD_ROW = new Text("good_row"); - private final Text PAST_STOP_ROW = new Text("zzzzzz"); + private final byte [] STOP_ROW = Bytes.toBytes("stop_row"); + private final byte [] GOOD_ROW = Bytes.toBytes("good_row"); + private final byte [] PAST_STOP_ROW = Bytes.toBytes("zzzzzz"); RowFilterInterface mainFilter; diff --git a/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java b/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java index 58aadb5424a..130810a3aea 100644 --- a/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java +++ b/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java @@ -24,9 +24,10 @@ import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; +import org.apache.hadoop.hbase.util.Bytes; + import junit.framework.TestCase; -import org.apache.hadoop.io.Text; /** * Tests for the while-match filter @@ -41,7 +42,7 @@ public class TestWhileMatchRowFilter extends TestCase { protected void setUp() throws Exception { super.setUp(); wmStopRowFilter = new WhileMatchRowFilter(new StopRowFilter( - new Text("s"))); + Bytes.toBytes("s"))); wmRegExpRowFilter = new WhileMatchRowFilter(new RegExpRowFilter( ".*regex.*")); } @@ -90,14 +91,14 @@ public class TestWhileMatchRowFilter extends TestCase { // Test cases that should pass the row toTest = "apples"; - assertFalse("filter: '" + toTest + "'", filter.filterRowKey(new Text(toTest))); - assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(new Text( + assertFalse("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest))); + assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes( toTest))); // Test cases that should fail the row toTest = "tuna"; - assertTrue("filter: '" + toTest + "'", filter.filterRowKey(new Text(toTest))); - assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(new Text( + assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest))); + assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes( toTest))); // The difference in switch @@ -121,14 +122,14 @@ public class TestWhileMatchRowFilter extends TestCase { // Test cases that should pass the row toTest = "regex_match"; - assertFalse("filter: '" + toTest + "'", filter.filterRowKey(new Text(toTest))); - assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(new Text( + assertFalse("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest))); + assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes( toTest))); // Test cases that should fail the row toTest = "not_a_match"; - assertTrue("filter: '" + toTest + "'", filter.filterRowKey(new Text(toTest))); - assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(new Text( + assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest))); + assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes( toTest))); // The difference in switch @@ -143,7 +144,7 @@ public class TestWhileMatchRowFilter extends TestCase { // Test filter(Text, Text, byte[]) for functionality only (no switch-cases) toTest = "asdf_regex_hjkl"; - assertFalse("filter: '" + toTest + "'", filter.filterColumn(new Text(toTest), + assertFalse("filter: '" + toTest + "'", filter.filterColumn(Bytes.toBytes(toTest), null, null)); } } diff --git a/src/test/org/apache/hadoop/hbase/hql/TestHQL.java b/src/test/org/apache/hadoop/hbase/hql/TestHQL.java deleted file mode 100644 index 863c53519af..00000000000 --- a/src/test/org/apache/hadoop/hbase/hql/TestHQL.java +++ /dev/null @@ -1,176 +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.hql; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.OutputStreamWriter; -import java.io.PrintStream; -import java.io.UnsupportedEncodingException; -import java.io.Writer; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.HBaseClusterTestCase; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.hql.generated.ParseException; -import org.apache.hadoop.hbase.hql.generated.HQLParser; -import org.apache.hadoop.io.Text; - -/** - * Tests for HQL - */ -public class TestHQL extends HBaseClusterTestCase { - - protected final Log LOG = LogFactory.getLog(this.getClass().getName()); - private ByteArrayOutputStream baos; - private HBaseAdmin admin; - - /** constructor */ - public TestHQL() { - super(1 /*One region server only*/); - } - - /** {@inheritDoc} */ - @Override - public void setUp() throws Exception { - super.setUp(); - // Capture System.out so we can grep for stuff in it. Have to do it once - // only because ConsoleTable sets up STDOUT in a static initialization - this.baos = new ByteArrayOutputStream(); - System.setOut(new PrintStream(this.baos)); - this.admin = new HBaseAdmin(this.conf); - } - - /** - * Create and then drop a table. - * Tests also that I can use single or double quotes around table and - * column family names. - * @throws Exception - */ - public void testCreateDeleteTable() throws Exception { - final String tableName = getName(); - final String columnFamily = tableName; - // Create table - createTable("create table " + tableName + " (" + columnFamily + ");", - tableName, columnFamily); - // Try describe - runCommand("describe " + tableName + ";"); - // Try describe with single quotes - runCommand("describe '" + tableName + "';"); - // Try describe with double-quotes - runCommand("describe \"" + tableName + "\";"); - // Try dropping the table. - dropTable("drop table " + tableName + ";", tableName); - // Use double-quotes creating table. - final String dblQuoteSuffix = "DblQuote"; - final String dblQuotedTableName = tableName + dblQuoteSuffix; - createTable("create table \"" + dblQuotedTableName + "\" (" + - columnFamily + ");", dblQuotedTableName, columnFamily); - // Use single-quotes creating table. - final String sglQuoteSuffix = "SglQuote"; - final String snglQuotedTableName = tableName + sglQuoteSuffix; - createTable("create table '" + snglQuotedTableName + "' (" + - columnFamily + ");", snglQuotedTableName, columnFamily); - // Use double-quotes around columnfamily name. - final String dblQuotedColumnFamily = columnFamily + dblQuoteSuffix; - String tmpTableName = tableName + dblQuotedColumnFamily; - createTable("create table " + tmpTableName + " (\"" + - dblQuotedColumnFamily + "\");", tmpTableName, - dblQuotedColumnFamily); - // Use single-quotes around columnfamily name. - final String sglQuotedColumnFamily = columnFamily + sglQuoteSuffix; - tmpTableName = tableName + sglQuotedColumnFamily; - createTable("create table " + tmpTableName + " ('" + - sglQuotedColumnFamily + "');", tmpTableName, sglQuotedColumnFamily); - } - - /** - * @throws Exception - */ - public void testInsertSelectDelete() throws Exception { - final String tableName = getName(); - final String columnFamily = tableName; - createTable("create table " + tableName + " (" + columnFamily + ");", - tableName, columnFamily); - // TODO: Add asserts that inserts, selects and deletes worked. - runCommand("insert into " + tableName + " (" + columnFamily + - ") values ('" + columnFamily + "') where row='" + columnFamily + "';"); - // Insert with double-quotes on row. - runCommand("insert into " + tableName + " (" + columnFamily + - ") values ('" + columnFamily + "') where row=\"" + columnFamily + "\";"); - // Insert with double-quotes on row and value. - runCommand("insert into " + tableName + " (" + columnFamily + - ") values (\"" + columnFamily + "\") where row=\"" + columnFamily + - "\";"); - runCommand("select \"" + columnFamily + "\" from \"" + tableName + - "\" where row=\"" + columnFamily + "\";"); - runCommand("delete \"" + columnFamily + ":\" from \"" + tableName + - "\" where row=\"" + columnFamily + "\";"); - } - - private void createTable(final String cmdStr, final String tableName, - final String columnFamily) - throws ParseException, IOException { - // Run create command. - runCommand(cmdStr); - // Assert table was created. - assertTrue(this.admin.tableExists(new Text(tableName))); - HTableDescriptor [] tables = this.admin.listTables(); - HTableDescriptor td = null; - for (int i = 0; i < tables.length; i++) { - if (tableName.equals(tables[i].getName().toString())) { - td = tables[i]; - } - } - assertNotNull(td); - assertTrue(td.hasFamily(new Text(columnFamily + ":"))); - } - - private void dropTable(final String cmdStr, final String tableName) - throws ParseException, IOException { - runCommand(cmdStr); - // Assert its gone - HTableDescriptor [] tables = this.admin.listTables(); - for (int i = 0; i < tables.length; i++) { - assertNotSame(tableName, tables[i].getName().toString()); - } - } - - private ReturnMsg runCommand(final String cmdStr) - throws ParseException, UnsupportedEncodingException { - LOG.info("Running command: " + cmdStr); - Writer out = new OutputStreamWriter(System.out, "UTF-8"); - TableFormatterFactory tff = new TableFormatterFactory(out, this.conf); - HQLParser parser = new HQLParser(cmdStr, out, tff.get()); - Command cmd = parser.terminatedCommand(); - ReturnMsg rm = cmd.execute(this.conf); - dumpStdout(); - return rm; - } - - private void dumpStdout() throws UnsupportedEncodingException { - LOG.info("STDOUT: " + - new String(this.baos.toByteArray(), HConstants.UTF8_ENCODING)); - this.baos.reset(); - } -} diff --git a/src/test/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java b/src/test/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java index d8baa8f42f6..427532781a2 100644 --- a/src/test/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java +++ b/src/test/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java @@ -29,9 +29,11 @@ import java.io.IOException; import junit.framework.TestCase; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.filter.RowFilterInterface; import org.apache.hadoop.hbase.filter.StopRowFilter; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.WritableComparator; public class TestHbaseObjectWritable extends TestCase { @@ -49,6 +51,11 @@ public class TestHbaseObjectWritable extends TestCase { // Do primitive type final int COUNT = 101; assertTrue(doType(conf, COUNT, int.class).equals(COUNT)); + // Do array + final byte [] testing = "testing".getBytes(); + byte [] result = (byte [])doType(conf, testing, testing.getClass()); + assertTrue(WritableComparator.compareBytes(testing, 0, testing.length, + result, 0, result.length) == 0); // Do unsupported type. boolean exception = false; try { @@ -67,7 +74,7 @@ public class TestHbaseObjectWritable extends TestCase { obj = doType(conf, new Text(""), Text.class); assertTrue(obj instanceof Text); // Try type that should get transferred old fashion way. - obj = doType(conf, new StopRowFilter(new Text("")), + obj = doType(conf, new StopRowFilter(HConstants.EMPTY_BYTE_ARRAY), RowFilterInterface.class); assertTrue(obj instanceof StopRowFilter); } diff --git a/src/test/org/apache/hadoop/hbase/io/TestTextSequence.java b/src/test/org/apache/hadoop/hbase/io/TestTextSequence.java deleted file mode 100644 index 7fc2d8c2ac0..00000000000 --- a/src/test/org/apache/hadoop/hbase/io/TestTextSequence.java +++ /dev/null @@ -1,53 +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.io; - -import org.apache.hadoop.hbase.HBaseTestCase; -import org.apache.hadoop.io.Text; - -public class TestTextSequence extends HBaseTestCase { - - protected void setUp() throws Exception { - super.setUp(); - } - - protected void tearDown() throws Exception { - super.tearDown(); - } - - /** - * Test compares of TextSequences and of TextSequence to Text. - * @throws Exception - */ - public void testCompare() throws Exception { - final Text a = new Text("abcdef"); - final Text b = new Text("defghi"); - TextSequence as = new TextSequence(a, 3); - TextSequence bs = new TextSequence(b, 0, 3); - assertTrue(as.compareTo(bs) == 0); - assertTrue(as.equals(bs)); - // Test where one is a Text and other is a TextSequence - final Text family = new Text("abc:"); - final Text column = new Text(family.toString() + "qualifier"); - final TextSequence ts = new TextSequence(column, 0, family.getLength()); - assertTrue(ts.compareTo(family) == 0); - assertTrue(ts.equals(family)); - } -} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/mapred/TestTableIndex.java b/src/test/org/apache/hadoop/hbase/mapred/TestTableIndex.java index 7f5acdac102..1639f1f33c8 100644 --- a/src/test/org/apache/hadoop/hbase/mapred/TestTableIndex.java +++ b/src/test/org/apache/hadoop/hbase/mapred/TestTableIndex.java @@ -36,13 +36,13 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Scanner; import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MultiRegionTable; -import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.MiniMRCluster; @@ -63,12 +63,12 @@ public class TestTableIndex extends MultiRegionTable { static final String TABLE_NAME = "moretest"; static final String INPUT_COLUMN = "contents:"; - static final Text TEXT_INPUT_COLUMN = new Text(INPUT_COLUMN); + static final byte [] TEXT_INPUT_COLUMN = Bytes.toBytes(INPUT_COLUMN); static final String OUTPUT_COLUMN = "text:"; - static final Text TEXT_OUTPUT_COLUMN = new Text(OUTPUT_COLUMN); + static final byte [] TEXT_OUTPUT_COLUMN = Bytes.toBytes(OUTPUT_COLUMN); static final String ROWKEY_NAME = "key"; static final String INDEX_DIR = "testindex"; - private static final Text[] columns = { + private static final byte [][] columns = new byte [][] { TEXT_INPUT_COLUMN, TEXT_OUTPUT_COLUMN }; @@ -177,7 +177,7 @@ public class TestTableIndex extends MultiRegionTable { private void scanTable(boolean printResults) throws IOException { - HTable table = new HTable(conf, new Text(TABLE_NAME)); + HTable table = new HTable(conf, TABLE_NAME); Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW); try { @@ -185,7 +185,7 @@ public class TestTableIndex extends MultiRegionTable { if (printResults) { LOG.info("row: " + r.getRow()); } - for (Map.Entry e : r.entrySet()) { + for (Map.Entry e : r.entrySet()) { if (printResults) { LOG.info(" column: " + e.getKey() + " value: " + new String(e.getValue().getValue(), HConstants.UTF8_ENCODING)); @@ -201,7 +201,7 @@ public class TestTableIndex extends MultiRegionTable { // Force a cache flush for every online region to ensure that when the // scanner takes its snapshot, all the updates have made it into the cache. for (HRegion r : cluster.getRegionThreads().get(0).getRegionServer(). - getOnlineRegions().values()) { + getOnlineRegions()) { HRegionIncommon region = new HRegionIncommon(r); region.flushcache(); } @@ -228,7 +228,7 @@ public class TestTableIndex extends MultiRegionTable { throw new IOException("no index directory found"); } - HTable table = new HTable(conf, new Text(TABLE_NAME)); + HTable table = new HTable(conf, TABLE_NAME); scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW); IndexConfiguration indexConf = new IndexConfiguration(); @@ -240,7 +240,7 @@ public class TestTableIndex extends MultiRegionTable { int count = 0; for (RowResult r : scanner) { - String value = r.getRow().toString(); + String value = Bytes.toString(r.getRow()); Term term = new Term(rowkeyName, value); int hitCount = searcher.search(new TermQuery(term)).length(); assertEquals("check row " + value, 1, hitCount); @@ -263,4 +263,4 @@ public class TestTableIndex extends MultiRegionTable { public static void main(@SuppressWarnings("unused") String[] args) { TestRunner.run(new TestSuite(TestTableIndex.class)); } -} +} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java b/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java index 2d42be89212..7ac4c0e5f46 100644 --- a/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java +++ b/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java @@ -19,8 +19,8 @@ */ package org.apache.hadoop.hbase.mapred; -import java.io.IOException; import java.io.File; +import java.io.IOException; import java.util.Map; import org.apache.commons.logging.Log; @@ -28,19 +28,20 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Scanner; -import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MultiRegionTable; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Scanner; import org.apache.hadoop.hbase.io.BatchUpdate; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.MiniMRCluster; -import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hadoop.mapred.Reporter; /** * Test Map/Reduce job over HBase tables. The map/reduce process we're testing @@ -54,13 +55,11 @@ public class TestTableMapReduce extends MultiRegionTable { static final String MULTI_REGION_TABLE_NAME = "mrtest"; static final String INPUT_COLUMN = "contents:"; - static final Text TEXT_INPUT_COLUMN = new Text(INPUT_COLUMN); static final String OUTPUT_COLUMN = "text:"; - static final Text TEXT_OUTPUT_COLUMN = new Text(OUTPUT_COLUMN); - private static final Text[] columns = { - TEXT_INPUT_COLUMN, - TEXT_OUTPUT_COLUMN + private static final byte [][] columns = new byte [][] { + Bytes.toBytes(INPUT_COLUMN), + Bytes.toBytes(OUTPUT_COLUMN) }; /** constructor */ @@ -74,22 +73,21 @@ public class TestTableMapReduce extends MultiRegionTable { /** * Pass the given key and processed record reduce */ - public static class ProcessContentsMapper extends TableMap { + public static class ProcessContentsMapper extends TableMap { /** * Pass the key, and reversed value to reduce */ @SuppressWarnings("unchecked") @Override - public void map(Text key, RowResult value, - OutputCollector output, + public void map(ImmutableBytesWritable key, RowResult value, + OutputCollector output, @SuppressWarnings("unused") Reporter reporter) throws IOException { - if(value.size() != 1) { + if (value.size() != 1) { throw new IOException("There should only be one input column"); } - - Text[] keys = value.keySet().toArray(new Text[value.size()]); - if(!keys[0].equals(TEXT_INPUT_COLUMN)) { + byte [][] keys = value.keySet().toArray(new byte [value.size()][]); + if(!Bytes.equals(keys[0], Bytes.toBytes(INPUT_COLUMN))) { throw new IOException("Wrong input column. Expected: " + INPUT_COLUMN + " but got: " + keys[0]); } @@ -105,10 +103,8 @@ public class TestTableMapReduce extends MultiRegionTable { // Now set the value to be collected - BatchUpdate outval = new BatchUpdate(key); - outval.put(TEXT_OUTPUT_COLUMN, - newValue.toString().getBytes(HConstants.UTF8_ENCODING)); - + BatchUpdate outval = new BatchUpdate(key.get()); + outval.put(OUTPUT_COLUMN, Bytes.toBytes(newValue.toString())); output.collect(key, outval); } } @@ -118,10 +114,9 @@ public class TestTableMapReduce extends MultiRegionTable { * @throws IOException */ public void testMultiRegionTable() throws IOException { - runTestOnTable(new HTable(conf, new Text(MULTI_REGION_TABLE_NAME))); + runTestOnTable(new HTable(conf, MULTI_REGION_TABLE_NAME)); } - private void runTestOnTable(HTable table) throws IOException { @SuppressWarnings("deprecation") MiniMRCluster mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1); @@ -132,19 +127,18 @@ public class TestTableMapReduce extends MultiRegionTable { jobConf = new JobConf(conf, TestTableMapReduce.class); jobConf.setJobName("process column contents"); jobConf.setNumReduceTasks(1); - - TableMap.initJob(table.getTableName().toString(), INPUT_COLUMN, - ProcessContentsMapper.class, Text.class, BatchUpdate.class, jobConf); - - TableReduce.initJob(table.getTableName().toString(), - IdentityTableReduce.class, jobConf); + TableMap.initJob(Bytes.toString(table.getTableName()), INPUT_COLUMN, + ProcessContentsMapper.class, ImmutableBytesWritable.class, + BatchUpdate.class, jobConf); + TableReduce.initJob(Bytes.toString(table.getTableName()), + IdentityTableReduce.class, jobConf); LOG.info("Started " + table.getTableName()); JobClient.runJob(jobConf); LOG.info("After map/reduce completion"); // verify map-reduce results - verify(table.getTableName().toString()); + verify(Bytes.toString(table.getTableName())); } finally { mrCluster.shutdown(); if (jobConf != null) { @@ -155,7 +149,7 @@ public class TestTableMapReduce extends MultiRegionTable { @SuppressWarnings("null") private void verify(String tableName) throws IOException { - HTable table = new HTable(conf, new Text(tableName)); + HTable table = new HTable(conf, tableName); boolean verified = false; long pause = conf.getLong("hbase.client.pause", 5 * 1000); int numRetries = conf.getInt("hbase.client.retries.number", 5); @@ -200,7 +194,7 @@ public class TestTableMapReduce extends MultiRegionTable { byte[] firstValue = null; byte[] secondValue = null; int count = 0; - for(Map.Entry e: r.entrySet()) { + for(Map.Entry e: r.entrySet()) { if (count == 0) { firstValue = e.getValue().getValue(); } @@ -215,14 +209,14 @@ public class TestTableMapReduce extends MultiRegionTable { String first = ""; if (firstValue == null) { - throw new NullPointerException(r.getRow().toString() + + throw new NullPointerException(Bytes.toString(r.getRow()) + ": first value is null"); } first = new String(firstValue, HConstants.UTF8_ENCODING); String second = ""; if (secondValue == null) { - throw new NullPointerException(r.getRow().toString() + + throw new NullPointerException(Bytes.toString(r.getRow()) + ": second value is null"); } byte[] secondReversed = new byte[secondValue.length]; diff --git a/src/test/org/apache/hadoop/hbase/regionserver/OOMERegionServer.java b/src/test/org/apache/hadoop/hbase/regionserver/OOMERegionServer.java index 8c7bc3d2cdf..3bc65fe9283 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/OOMERegionServer.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/OOMERegionServer.java @@ -26,7 +26,6 @@ import java.util.List; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HServerAddress; -import org.apache.hadoop.io.Text; /** * A region server that will OOME. @@ -47,7 +46,7 @@ public class OOMERegionServer extends HRegionServer { super(address, conf); } - public void batchUpdate(Text regionName, BatchUpdate b) + public void batchUpdate(byte [] regionName, BatchUpdate b) throws IOException { super.batchUpdate(regionName, b); for (int i = 0; i < 30; i++) { diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/src/test/org/apache/hadoop/hbase/regionserver/TestCompaction.java index f62f7586c9d..74d153ef6b5 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestCompaction.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestCompaction.java @@ -25,15 +25,15 @@ import org.apache.hadoop.dfs.MiniDFSCluster; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.io.MapFile; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HStoreKey; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.util.Bytes; /** * Test compactions @@ -41,11 +41,11 @@ import org.apache.hadoop.hbase.io.Cell; public class TestCompaction extends HBaseTestCase { static final Log LOG = LogFactory.getLog(TestCompaction.class.getName()); private HRegion r = null; - private static final String COLUMN_FAMILY = COLFAMILY_NAME1; - 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 byte [] COLUMN_FAMILY = COLFAMILY_NAME1; + private final byte [] STARTROW = Bytes.toBytes(START_KEY); + private static final byte [] COLUMN_FAMILY_TEXT = COLUMN_FAMILY; + private static final byte [] COLUMN_FAMILY_TEXT_MINUS_COLON = + Bytes.toBytes(Bytes.toString(COLUMN_FAMILY).substring(0, COLUMN_FAMILY.length - 1)); private static final int COMPACTION_THRESHOLD = MAXVERSIONS; private MiniDFSCluster cluster; @@ -53,7 +53,6 @@ public class TestCompaction extends HBaseTestCase { /** constructor */ public TestCompaction() { super(); - STARTROW = new Text(START_KEY); // Set cache flush size to 1MB conf.setInt("hbase.hregion.memcache.flush.size", 1024*1024); @@ -99,7 +98,7 @@ public class TestCompaction extends HBaseTestCase { // Default is that there only 3 (MAXVERSIONS) versions allowed per column. // Assert > 3 and then after compaction, assert that only 3 versions // available. - addContent(new HRegionIncommon(r), COLUMN_FAMILY); + addContent(new HRegionIncommon(r), Bytes.toString(COLUMN_FAMILY)); Cell[] cellValues = r.get(STARTROW, COLUMN_FAMILY_TEXT, 100 /*Too many*/); // Assert that I can get > 5 versions (Should be at least 5 in there). @@ -114,9 +113,8 @@ public class TestCompaction extends HBaseTestCase { 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); - cellValues = r.get(secondRow, COLUMN_FAMILY_TEXT, 100/*Too many*/); - LOG.info("Count of " + secondRow + ": " + cellValues.length); + cellValues = r.get(secondRowBytes, COLUMN_FAMILY_TEXT, 100/*Too many*/); + LOG.info("Count of " + Bytes.toString(secondRowBytes) + ": " + cellValues.length); // Commented out because fails on an hp+ubuntu single-processor w/ 1G and // "Intel(R) Pentium(R) 4 CPU 3.20GHz" though passes on all local // machines and even on hudson. On said machine, its reporting in the @@ -147,26 +145,26 @@ public class TestCompaction extends HBaseTestCase { cellValues = r.get(STARTROW, COLUMN_FAMILY_TEXT, 100 /*Too many*/); assertNull(cellValues); // Assert the store files do not have the first record 'aaa' keys in them. - for (MapFile.Reader reader: - this.r.stores.get(COLUMN_FAMILY_TEXT_MINUS_COLON).getReaders()) { + for (MapFile.Reader reader: this.r.stores. + get(Bytes.mapKey(COLUMN_FAMILY_TEXT_MINUS_COLON)).getReaders()) { reader.reset(); HStoreKey key = new HStoreKey(); ImmutableBytesWritable val = new ImmutableBytesWritable(); while(reader.next(key, val)) { - assertFalse(key.getRow().equals(STARTROW)); + assertFalse(Bytes.equals(key.getRow(), STARTROW)); } } } private void createStoreFile(final HRegion region) throws IOException { HRegionIncommon loader = new HRegionIncommon(region); - addContent(loader, COLUMN_FAMILY); + addContent(loader, Bytes.toString(COLUMN_FAMILY)); loader.flushcache(); } private void createSmallerStoreFile(final HRegion region) throws IOException { HRegionIncommon loader = new HRegionIncommon(region); - addContent(loader, COLUMN_FAMILY, + addContent(loader, Bytes.toString(COLUMN_FAMILY), ("bbb" + PUNCTUATION).getBytes(), null); loader.flushcache(); } diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java b/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java index 8977cc7a42e..b8c9e15c061 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java @@ -19,17 +19,14 @@ */ package org.apache.hadoop.hbase.regionserver; -import java.io.IOException; -import java.util.Map; - +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.dfs.MiniDFSCluster; -import org.apache.hadoop.io.Text; -import org.apache.commons.logging.*; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.util.Bytes; /** * Test the functionality of deleteAll. @@ -85,15 +82,15 @@ public class TestDeleteAll extends HBaseTestCase { boolean flush) throws Exception{ // insert a few versions worth of data for a row - Text row = new Text("test_row"); + byte [] row = Bytes.toBytes("test_row"); long t0 = System.currentTimeMillis(); long t1 = t0 - 15000; long t2 = t1 - 15000; - Text colA = new Text(COLUMNS[0].toString() + "a"); - Text colB = new Text(COLUMNS[0].toString() + "b"); - Text colC = new Text(COLUMNS[0].toString() + "c"); - Text colD = new Text(COLUMNS[0].toString()); + byte [] colA = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "a"); + byte [] colB = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "b"); + byte [] colC = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "c"); + byte [] colD = Bytes.toBytes(Bytes.toString(COLUMNS[0])); BatchUpdate batchUpdate = new BatchUpdate(row, t0); batchUpdate.put(colA, cellData(0, flush).getBytes()); diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteFamily.java b/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteFamily.java index 2f2881f26de..43fbf6a089c 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteFamily.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteFamily.java @@ -19,19 +19,15 @@ */ package org.apache.hadoop.hbase.regionserver; -import java.io.IOException; -import java.util.Map; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.dfs.MiniDFSCluster; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.HBaseTestCase; - import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.util.Bytes; /** * Test the functionality of deleteFamily. @@ -82,14 +78,14 @@ public class TestDeleteFamily extends HBaseTestCase { boolean flush) throws Exception{ // insert a few versions worth of data for a row - Text row = new Text("test_row"); + byte [] row = Bytes.toBytes("test_row"); long t0 = System.currentTimeMillis(); long t1 = t0 - 15000; long t2 = t1 - 15000; - Text colA = new Text(COLUMNS[0].toString() + "a"); - Text colB = new Text(COLUMNS[0].toString() + "b"); - Text colC = new Text(COLUMNS[1].toString() + "c"); + byte [] colA = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "a"); + byte [] colB = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "b"); + byte [] colC = Bytes.toBytes(Bytes.toString(COLUMNS[1]) + "c"); BatchUpdate batchUpdate = null; batchUpdate = new BatchUpdate(row, t0); diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestGet.java b/src/test/org/apache/hadoop/hbase/regionserver/TestGet.java index ec74f18f08c..5bc6a879b54 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestGet.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestGet.java @@ -29,6 +29,7 @@ import org.apache.hadoop.dfs.MiniDFSCluster; import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.HBaseTestCase; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.hbase.HRegionInfo; @@ -43,9 +44,9 @@ import org.apache.hadoop.hbase.io.BatchUpdate; public class TestGet extends HBaseTestCase { private static final Log LOG = LogFactory.getLog(TestGet.class.getName()); - private static final Text CONTENTS = new Text("contents:"); - private static final Text ROW_KEY = - new Text(HRegionInfo.rootRegionInfo.getRegionName()); + private static final byte [] CONTENTS = Bytes.toBytes("contents:"); + private static final byte [] ROW_KEY = + HRegionInfo.ROOT_REGIONINFO.getRegionName(); private static final String SERVER_ADDRESS = "foo.bar.com:1234"; @@ -61,12 +62,12 @@ public class TestGet extends HBaseTestCase { assertNull(value); // Find out what getFull returns - Map values = r.getFull(ROW_KEY); + Map values = r.getFull(ROW_KEY); // assertEquals(4, values.keySet().size()); - for(Iterator i = values.keySet().iterator(); i.hasNext(); ) { - Text column = i.next(); - if (column.equals(HConstants.COL_SERVER)) { + for(Iterator i = values.keySet().iterator(); i.hasNext(); ) { + byte [] column = i.next(); + if (Bytes.equals(column, HConstants.COL_SERVER)) { String server = Writables.cellToString(values.get(column)); assertEquals(expectedServer, server); LOG.info(server); @@ -92,8 +93,8 @@ public class TestGet extends HBaseTestCase { cluster.getFileSystem().getHomeDirectory().toString()); HTableDescriptor desc = new HTableDescriptor("test"); - desc.addFamily(new HColumnDescriptor(CONTENTS.toString())); - desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString())); + desc.addFamily(new HColumnDescriptor(CONTENTS)); + desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY)); region = createNewHRegion(desc, null, null); HRegionIncommon r = new HRegionIncommon(region); @@ -102,17 +103,17 @@ public class TestGet extends HBaseTestCase { BatchUpdate batchUpdate = null; batchUpdate = new BatchUpdate(ROW_KEY, System.currentTimeMillis()); - batchUpdate.put(CONTENTS, Writables.getBytes(CONTENTS)); + batchUpdate.put(CONTENTS, CONTENTS); batchUpdate.put(HConstants.COL_REGIONINFO, - Writables.getBytes(HRegionInfo.rootRegionInfo)); + Writables.getBytes(HRegionInfo.ROOT_REGIONINFO)); r.commit(batchUpdate); batchUpdate = new BatchUpdate(ROW_KEY, System.currentTimeMillis()); batchUpdate.put(HConstants.COL_SERVER, - Writables.stringToBytes(new HServerAddress(SERVER_ADDRESS).toString())); - batchUpdate.put(HConstants.COL_STARTCODE, Writables.longToBytes(12345)); - batchUpdate.put(new Text(HConstants.COLUMN_FAMILY + "region"), - "region".getBytes(HConstants.UTF8_ENCODING)); + Bytes.toBytes(new HServerAddress(SERVER_ADDRESS).toString())); + batchUpdate.put(HConstants.COL_STARTCODE, Bytes.toBytes(12345)); + batchUpdate.put(new Text(Bytes.toString(HConstants.COLUMN_FAMILY) + + "region"), Bytes.toBytes("region")); r.commit(batchUpdate); // Verify that get works the same from memcache as when reading from disk @@ -133,12 +134,14 @@ public class TestGet extends HBaseTestCase { // Update one family member and add a new one batchUpdate = new BatchUpdate(ROW_KEY, System.currentTimeMillis()); - batchUpdate.put(new Text(HConstants.COLUMN_FAMILY + "region"), + batchUpdate.put(new Text(Bytes.toString(HConstants.COLUMN_FAMILY) + + "region"), "region2".getBytes(HConstants.UTF8_ENCODING)); String otherServerName = "bar.foo.com:4321"; batchUpdate.put(HConstants.COL_SERVER, - Writables.stringToBytes(new HServerAddress(otherServerName).toString())); - batchUpdate.put(new Text(HConstants.COLUMN_FAMILY + "junk"), + Bytes.toBytes(new HServerAddress(otherServerName).toString())); + batchUpdate.put(new Text(Bytes.toString(HConstants.COLUMN_FAMILY) + + "junk"), "junk".getBytes(HConstants.UTF8_ENCODING)); r.commit(batchUpdate); diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestGet2.java b/src/test/org/apache/hadoop/hbase/regionserver/TestGet2.java index e22d2814e9f..0b7c089d6b2 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestGet2.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestGet2.java @@ -24,17 +24,18 @@ import java.util.Map; import java.util.HashSet; import java.util.TreeMap; import java.util.Set; +import java.util.TreeSet; import org.apache.hadoop.dfs.MiniDFSCluster; import org.apache.hadoop.hbase.filter.StopRowFilter; import org.apache.hadoop.hbase.filter.WhileMatchRowFilter; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HStoreKey; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.util.Bytes; /** * {@link TestGet} is a medley of tests of get all done up as a single test. @@ -63,22 +64,23 @@ public class TestGet2 extends HBaseTestCase implements HConstants { HTableDescriptor htd = createTableDescriptor(getName()); region = createNewHRegion(htd, null, null); for (int i = 0; i < COLUMNS.length; i++) { - addContent(region, COLUMNS[i].toString()); + addContent(region, COLUMNS[i]); } // Find two rows to use doing getFull. - final Text arbitraryStartRow = new Text("b"); - Text actualStartRow = null; - final Text arbitraryStopRow = new Text("c"); - Text actualStopRow = null; - Text [] columns = new Text [] {new Text(COLFAMILY_NAME1)}; + final byte [] arbitraryStartRow = Bytes.toBytes("b"); + byte [] actualStartRow = null; + final byte [] arbitraryStopRow = Bytes.toBytes("c"); + byte [] actualStopRow = null; + byte [][] columns = {COLFAMILY_NAME1}; scanner = region.getScanner(columns, arbitraryStartRow, HConstants.LATEST_TIMESTAMP, new WhileMatchRowFilter(new StopRowFilter(arbitraryStopRow))); HStoreKey key = new HStoreKey(); - TreeMap value = new TreeMap(); + TreeMap value = + new TreeMap(Bytes.BYTES_COMPARATOR); while (scanner.next(key, value)) { if (actualStartRow == null) { - actualStartRow = new Text(key.getRow()); + actualStartRow = key.getRow(); } else { actualStopRow = key.getRow(); } @@ -120,7 +122,7 @@ public class TestGet2 extends HBaseTestCase implements HConstants { long right_now = System.currentTimeMillis(); long one_second_ago = right_now - 1000; - Text t = new Text("test_row"); + String t = "test_row"; BatchUpdate batchUpdate = new BatchUpdate(t, one_second_ago); batchUpdate.put(COLUMNS[0], "old text".getBytes()); region_incommon.commit(batchUpdate); @@ -129,14 +131,16 @@ public class TestGet2 extends HBaseTestCase implements HConstants { batchUpdate.put(COLUMNS[0], "new text".getBytes()); region_incommon.commit(batchUpdate); - assertCellEquals(region, t, COLUMNS[0], right_now, "new text"); - assertCellEquals(region, t, COLUMNS[0], one_second_ago, "old text"); + assertCellEquals(region, Bytes.toBytes(t), COLUMNS[0], + right_now, "new text"); + assertCellEquals(region, Bytes.toBytes(t), COLUMNS[0], + one_second_ago, "old text"); // Force a flush so store files come into play. region_incommon.flushcache(); - assertCellEquals(region, t, COLUMNS[0], right_now, "new text"); - assertCellEquals(region, t, COLUMNS[0], one_second_ago, "old text"); + assertCellEquals(region, Bytes.toBytes(t), COLUMNS[0], right_now, "new text"); + assertCellEquals(region, Bytes.toBytes(t), COLUMNS[0], one_second_ago, "old text"); } finally { if (region != null) { @@ -166,11 +170,11 @@ public class TestGet2 extends HBaseTestCase implements HConstants { region_incommon = new HRegionIncommon(region); // set up some test data - Text t10 = new Text("010"); - Text t20 = new Text("020"); - Text t30 = new Text("030"); - Text t35 = new Text("035"); - Text t40 = new Text("040"); + String t10 = "010"; + String t20 = "020"; + String t30 = "030"; + String t35 = "035"; + String t40 = "040"; batchUpdate = new BatchUpdate(t10); batchUpdate.put(COLUMNS[0], "t10 bytes".getBytes()); @@ -197,42 +201,42 @@ public class TestGet2 extends HBaseTestCase implements HConstants { region.batchUpdate(batchUpdate); // try finding "015" - Text t15 = new Text("015"); - Map results = - region.getClosestRowBefore(t15); + String t15 = "015"; + Map results = + region.getClosestRowBefore(Bytes.toBytes(t15)); assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t10 bytes"); // try "020", we should get that row exactly - results = region.getClosestRowBefore(t20); + results = region.getClosestRowBefore(Bytes.toBytes(t20)); assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t20 bytes"); // try "038", should skip deleted "035" and get "030" - Text t38 = new Text("038"); - results = region.getClosestRowBefore(t38); + String t38 = "038"; + results = region.getClosestRowBefore(Bytes.toBytes(t38)); assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes"); // try "050", should get stuff from "040" - Text t50 = new Text("050"); - results = region.getClosestRowBefore(t50); + String t50 = "050"; + results = region.getClosestRowBefore(Bytes.toBytes(t50)); assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t40 bytes"); // force a flush region.flushcache(); // try finding "015" - results = region.getClosestRowBefore(t15); + results = region.getClosestRowBefore(Bytes.toBytes(t15)); assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t10 bytes"); // try "020", we should get that row exactly - results = region.getClosestRowBefore(t20); + results = region.getClosestRowBefore(Bytes.toBytes(t20)); assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t20 bytes"); // try "038", should skip deleted "035" and get "030" - results = region.getClosestRowBefore(t38); + results = region.getClosestRowBefore(Bytes.toBytes(t38)); assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes"); // try "050", should get stuff from "040" - results = region.getClosestRowBefore(t50); + results = region.getClosestRowBefore(Bytes.toBytes(t50)); assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t40 bytes"); } finally { if (region != null) { @@ -258,7 +262,7 @@ public class TestGet2 extends HBaseTestCase implements HConstants { region_incommon = new HRegionIncommon(region); // write a row with a bunch of columns - Text row = new Text("some_row"); + byte [] row = Bytes.toBytes("some_row"); BatchUpdate bu = new BatchUpdate(row); bu.put(COLUMNS[0], "column 0".getBytes()); bu.put(COLUMNS[1], "column 1".getBytes()); @@ -283,11 +287,11 @@ public class TestGet2 extends HBaseTestCase implements HConstants { } } - private void assertSpecifiedColumns(final HRegion region, final Text row) + private void assertSpecifiedColumns(final HRegion region, final byte [] row) throws IOException { - HashSet all = new HashSet(); - HashSet one = new HashSet(); - HashSet none = new HashSet(); + TreeSet all = new TreeSet(Bytes.BYTES_COMPARATOR); + TreeSet one = new TreeSet(Bytes.BYTES_COMPARATOR); + TreeSet none = new TreeSet(Bytes.BYTES_COMPARATOR); all.add(COLUMNS[0]); all.add(COLUMNS[1]); @@ -295,7 +299,7 @@ public class TestGet2 extends HBaseTestCase implements HConstants { one.add(COLUMNS[0]); // make sure we get all of them with standard getFull - Map result = region.getFull(row, null, + Map result = region.getFull(row, null, HConstants.LATEST_TIMESTAMP); assertEquals(new String(result.get(COLUMNS[0]).getValue()), "column 0"); assertEquals(new String(result.get(COLUMNS[1]).getValue()), "column 1"); @@ -324,7 +328,7 @@ public class TestGet2 extends HBaseTestCase implements HConstants { HRegion region = null; HRegionIncommon region_incommon = null; BatchUpdate batchUpdate = null; - Map results = null; + Map results = null; try { HTableDescriptor htd = createTableDescriptor(getName()); @@ -334,7 +338,7 @@ public class TestGet2 extends HBaseTestCase implements HConstants { // // Test ordering issue // - Text row = new Text("row1"); + byte [] row = Bytes.toBytes("row1"); // write some data batchUpdate = new BatchUpdate(row); @@ -345,7 +349,7 @@ public class TestGet2 extends HBaseTestCase implements HConstants { region.flushcache(); // assert that getFull gives us the older value - results = region.getFull(row, (Set)null, LATEST_TIMESTAMP); + results = region.getFull(row, (Set)null, LATEST_TIMESTAMP); assertEquals("olderValue", new String(results.get(COLUMNS[0]).getValue())); // write a new value for the cell @@ -357,16 +361,16 @@ public class TestGet2 extends HBaseTestCase implements HConstants { region.flushcache(); // assert that getFull gives us the later value - results = region.getFull(row, (Set)null, LATEST_TIMESTAMP); + results = region.getFull(row, (Set)null, LATEST_TIMESTAMP); assertEquals("newerValue", new String(results.get(COLUMNS[0]).getValue())); // // Test the delete masking issue // - Text row2 = new Text("row2"); - Text cell1 = new Text(COLUMNS[0].toString() + "a"); - Text cell2 = new Text(COLUMNS[0].toString() + "b"); - Text cell3 = new Text(COLUMNS[0].toString() + "c"); + byte [] row2 = Bytes.toBytes("row2"); + byte [] cell1 = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "a"); + byte [] cell2 = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "b"); + byte [] cell3 = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "c"); // write some data at two columns batchUpdate = new BatchUpdate(row2); @@ -378,7 +382,7 @@ public class TestGet2 extends HBaseTestCase implements HConstants { region.flushcache(); // assert i get both columns - results = region.getFull(row2, (Set)null, LATEST_TIMESTAMP); + results = region.getFull(row2, (Set)null, LATEST_TIMESTAMP); assertEquals("Should have two columns in the results map", 2, results.size()); assertEquals("column0 value", new String(results.get(cell1).getValue())); assertEquals("column1 value", new String(results.get(cell2).getValue())); @@ -393,7 +397,7 @@ public class TestGet2 extends HBaseTestCase implements HConstants { region.flushcache(); // assert i get the second column only - results = region.getFull(row2, (Set)null, LATEST_TIMESTAMP); + results = region.getFull(row2, (Set)null, LATEST_TIMESTAMP); assertEquals("Should have one column in the results map", 1, results.size()); assertNull("column0 value", results.get(cell1)); assertEquals("column1 new value", new String(results.get(cell2).getValue())); @@ -407,7 +411,7 @@ public class TestGet2 extends HBaseTestCase implements HConstants { region.batchUpdate(batchUpdate); // assert i get the third column only - results = region.getFull(row2, (Set)null, LATEST_TIMESTAMP); + results = region.getFull(row2, (Set)null, LATEST_TIMESTAMP); assertEquals("Should have one column in the results map", 1, results.size()); assertNull("column0 value", results.get(cell1)); assertNull("column1 value", results.get(cell2)); @@ -425,18 +429,18 @@ public class TestGet2 extends HBaseTestCase implements HConstants { } } - private void assertColumnsPresent(final HRegion r, final Text row) + private void assertColumnsPresent(final HRegion r, final byte [] row) throws IOException { - Map result = r.getFull(row, null, HConstants.LATEST_TIMESTAMP); + Map result = r.getFull(row, null, HConstants.LATEST_TIMESTAMP); int columnCount = 0; - for (Map.Entry e: result.entrySet()) { + for (Map.Entry e: result.entrySet()) { columnCount++; - String column = e.getKey().toString(); + byte [] column = e.getKey(); boolean legitColumn = false; for (int i = 0; i < COLUMNS.length; i++) { // Assert value is same as row. This is 'nature' of the data added. - assertTrue(row.equals(new Text(e.getValue().getValue()))); - if (COLUMNS[i].equals(new Text(column))) { + assertTrue(Bytes.equals(row, e.getValue().getValue())); + if (Bytes.equals(COLUMNS[i], column)) { legitColumn = true; break; } diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java b/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java index 4dd48711fa0..2f536ec615d 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java @@ -25,12 +25,12 @@ import java.util.TreeMap; import org.apache.hadoop.dfs.MiniDFSCluster; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.io.Text; import org.apache.hadoop.io.SequenceFile.Reader; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HStoreKey; +import org.apache.hadoop.hbase.util.Bytes; /** JUnit test case for HLog */ public class TestHLog extends HBaseTestCase implements HConstants { @@ -67,8 +67,8 @@ public class TestHLog extends HBaseTestCase implements HConstants { * @throws IOException */ public void testSplit() throws IOException { - final Text tableName = new Text(getName()); - final Text rowName = tableName; + final byte [] tableName = Bytes.toBytes(getName()); + final byte [] rowName = tableName; HLog log = new HLog(this.fs, this.dir, this.conf, null); // Add edits for three regions. try { @@ -76,11 +76,10 @@ public class TestHLog extends HBaseTestCase implements HConstants { for (int i = 0; i < 3; i++) { for (int j = 0; j < 3; j++) { TreeMap edit = new TreeMap(); - Text column = new Text(Integer.toString(j)); - edit.put( - new HStoreKey(rowName, column, System.currentTimeMillis()), - column.getBytes()); - log.append(new Text(Integer.toString(i)), tableName, edit); + byte [] column = Bytes.toBytes(Integer.toString(j)); + edit.put(new HStoreKey(rowName, column, System.currentTimeMillis()), + column); + log.append(Bytes.toBytes(Integer.toString(i)), tableName, edit); } } log.rollWriter(); @@ -99,9 +98,9 @@ public class TestHLog extends HBaseTestCase implements HConstants { */ public void testAppend() throws IOException { final int COL_COUNT = 10; - final Text regionName = new Text("regionname"); - final Text tableName = new Text("tablename"); - final Text row = new Text("row"); + final byte [] regionName = Bytes.toBytes("regionname"); + final byte [] tableName = Bytes.toBytes("tablename"); + final byte [] row = Bytes.toBytes("row"); Reader reader = null; HLog log = new HLog(fs, dir, this.conf, null); try { @@ -110,7 +109,7 @@ public class TestHLog extends HBaseTestCase implements HConstants { long timestamp = System.currentTimeMillis(); TreeMap cols = new TreeMap(); for (int i = 0; i < COL_COUNT; i++) { - cols.put(new HStoreKey(row, new Text(Integer.toString(i)), timestamp), + cols.put(new HStoreKey(row, Bytes.toBytes(Integer.toString(i)), timestamp), new byte[] { (byte)(i + '0') }); } log.append(regionName, tableName, cols); @@ -125,18 +124,18 @@ public class TestHLog extends HBaseTestCase implements HConstants { HLogEdit val = new HLogEdit(); for (int i = 0; i < COL_COUNT; i++) { reader.next(key, val); - assertEquals(regionName, key.getRegionName()); - assertEquals(tableName, key.getTablename()); - assertEquals(row, key.getRow()); + assertTrue(Bytes.equals(regionName, key.getRegionName())); + assertTrue(Bytes.equals(tableName, key.getTablename())); + assertTrue(Bytes.equals(row, key.getRow())); assertEquals((byte)(i + '0'), val.getVal()[0]); System.out.println(key + " " + val); } while (reader.next(key, val)) { // Assert only one more row... the meta flushed row. - assertEquals(regionName, key.getRegionName()); - assertEquals(tableName, key.getTablename()); - assertEquals(HLog.METAROW, key.getRow()); - assertEquals(HLog.METACOLUMN, val.getColumn()); + assertTrue(Bytes.equals(regionName, key.getRegionName())); + assertTrue(Bytes.equals(tableName, key.getTablename())); + assertTrue(Bytes.equals(HLog.METAROW, key.getRow())); + assertTrue(Bytes.equals(HLog.METACOLUMN, val.getColumn())); assertEquals(0, HLogEdit.completeCacheFlush.compareTo(val.getVal())); System.out.println(key + " " + val); } diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestHMemcache.java b/src/test/org/apache/hadoop/hbase/regionserver/TestHMemcache.java index a48a8caae5a..665e9b4b7d1 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestHMemcache.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestHMemcache.java @@ -29,11 +29,10 @@ import java.util.TreeMap; import junit.framework.TestCase; -import org.apache.hadoop.io.Text; - import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HStoreKey; import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.util.Bytes; /** memcache test case */ public class TestHMemcache extends TestCase { @@ -48,7 +47,7 @@ public class TestHMemcache extends TestCase { private static final int FIRST_ROW = 1; private static final int NUM_VALS = 1000; - private static final Text CONTENTS_BASIC = new Text("contents:basic"); + private static final byte [] CONTENTS_BASIC = Bytes.toBytes("contents:basic"); private static final String CONTENTSTR = "contentstr"; private static final String ANCHORNUM = "anchor:anchornum-"; private static final String ANCHORSTR = "anchorstr"; @@ -65,13 +64,13 @@ public class TestHMemcache extends TestCase { */ public void testMemcache() throws UnsupportedEncodingException { for (int k = FIRST_ROW; k <= NUM_VALS; k++) { - Text row = new Text("row_" + k); + byte [] row = Bytes.toBytes("row_" + k); HStoreKey key = new HStoreKey(row, CONTENTS_BASIC, System.currentTimeMillis()); hmemcache.add(key, (CONTENTSTR + k).getBytes(HConstants.UTF8_ENCODING)); key = - new HStoreKey(row, new Text(ANCHORNUM + k), System.currentTimeMillis()); + new HStoreKey(row, Bytes.toBytes(ANCHORNUM + k), System.currentTimeMillis()); hmemcache.add(key, (ANCHORSTR + k).getBytes(HConstants.UTF8_ENCODING)); } @@ -79,7 +78,7 @@ public class TestHMemcache extends TestCase { for (int k = FIRST_ROW; k <= NUM_VALS; k++) { List results; - Text row = new Text("row_" + k); + byte [] row = Bytes.toBytes("row_" + k); HStoreKey key = new HStoreKey(row, CONTENTS_BASIC, Long.MAX_VALUE); results = hmemcache.get(key, 1); assertNotNull("no data for " + key.toString(), results); @@ -91,7 +90,7 @@ public class TestHMemcache extends TestCase { "), expected: '" + teststr + "' got: '" + bodystr + "'", teststr.compareTo(bodystr) == 0); - key = new HStoreKey(row, new Text(ANCHORNUM + k), Long.MAX_VALUE); + key = new HStoreKey(row, Bytes.toBytes(ANCHORNUM + k), Long.MAX_VALUE); results = hmemcache.get(key, 1); assertNotNull("no data for " + key.toString(), results); assertEquals(1, results.size()); @@ -104,12 +103,12 @@ public class TestHMemcache extends TestCase { } } - private Text getRowName(final int index) { - return new Text("row" + Integer.toString(index)); + private byte [] getRowName(final int index) { + return Bytes.toBytes("row" + Integer.toString(index)); } - private Text getColumnName(final int rowIndex, final int colIndex) { - return new Text(COLUMN_FAMILY + ":" + Integer.toString(rowIndex) + ";" + + private byte [] getColumnName(final int rowIndex, final int colIndex) { + return Bytes.toBytes(COLUMN_FAMILY + ":" + Integer.toString(rowIndex) + ";" + Integer.toString(colIndex)); } @@ -117,15 +116,12 @@ public class TestHMemcache extends TestCase { * Adds {@link #ROW_COUNT} rows and {@link #COLUMNS_COUNT} * @param hmc Instance to add rows to. */ - private void addRows(final Memcache hmc) - throws UnsupportedEncodingException { - + private void addRows(final Memcache hmc) { for (int i = 0; i < ROW_COUNT; i++) { long timestamp = System.currentTimeMillis(); for (int ii = 0; ii < COLUMNS_COUNT; ii++) { - Text k = getColumnName(i, ii); - hmc.add(new HStoreKey(getRowName(i), k, timestamp), - k.toString().getBytes(HConstants.UTF8_ENCODING)); + byte [] k = getColumnName(i, ii); + hmc.add(new HStoreKey(getRowName(i), k, timestamp), k); } } } @@ -155,27 +151,26 @@ public class TestHMemcache extends TestCase { } } - private void isExpectedRowWithoutTimestamps(final int rowIndex, TreeMap row) - throws UnsupportedEncodingException { + private void isExpectedRowWithoutTimestamps(final int rowIndex, TreeMap row) { int i = 0; - for (Text colname: row.keySet()) { - String expectedColname = getColumnName(rowIndex, i++).toString(); - String colnameStr = colname.toString(); + for (byte [] colname: row.keySet()) { + String expectedColname = Bytes.toString(getColumnName(rowIndex, i++)); + String colnameStr = Bytes.toString(colname); assertEquals("Column name", colnameStr, expectedColname); // Value is column name as bytes. Usually result is // 100 bytes in size at least. This is the default size // for BytesWriteable. For comparison, comvert bytes to // String and trim to remove trailing null bytes. byte [] value = row.get(colname); - String colvalueStr = new String(value, HConstants.UTF8_ENCODING).trim(); + String colvalueStr = Bytes.toString(value).trim(); assertEquals("Content", colnameStr, colvalueStr); } } - private void isExpectedRow(final int rowIndex, TreeMap row) - throws UnsupportedEncodingException { - TreeMap converted = new TreeMap(); - for (Map.Entry entry : row.entrySet()) { + private void isExpectedRow(final int rowIndex, TreeMap row) { + TreeMap converted = + new TreeMap(Bytes.BYTES_COMPARATOR); + for (Map.Entry entry : row.entrySet()) { converted.put(entry.getKey(), entry.getValue() == null ? null : entry.getValue().getValue()); } @@ -183,49 +178,49 @@ public class TestHMemcache extends TestCase { } /** Test getFull from memcache - * @throws UnsupportedEncodingException */ - public void testGetFull() throws UnsupportedEncodingException { + public void testGetFull() { addRows(this.hmemcache); for (int i = 0; i < ROW_COUNT; i++) { HStoreKey hsk = new HStoreKey(getRowName(i)); - TreeMap all = new TreeMap(); - TreeMap deletes = new TreeMap(); + TreeMap all = + new TreeMap(Bytes.BYTES_COMPARATOR); + TreeMap deletes = + new TreeMap(Bytes.BYTES_COMPARATOR); this.hmemcache.getFull(hsk, null, deletes, all); isExpectedRow(i, all); } } /** Test getNextRow from memcache - * @throws UnsupportedEncodingException */ - public void testGetNextRow() throws UnsupportedEncodingException { + public void testGetNextRow() { addRows(this.hmemcache); - Text closestToEmpty = this.hmemcache.getNextRow(HConstants.EMPTY_TEXT); - assertEquals(closestToEmpty, getRowName(0)); + byte [] closestToEmpty = + this.hmemcache.getNextRow(HConstants.EMPTY_BYTE_ARRAY); + assertTrue(Bytes.equals(closestToEmpty, getRowName(0))); for (int i = 0; i < ROW_COUNT; i++) { - Text nr = this.hmemcache.getNextRow(getRowName(i)); + byte [] nr = this.hmemcache.getNextRow(getRowName(i)); if (i + 1 == ROW_COUNT) { assertEquals(nr, null); } else { - assertEquals(nr, getRowName(i + 1)); + assertTrue(Bytes.equals(nr, getRowName(i + 1))); } } } /** Test getClosest from memcache - * @throws UnsupportedEncodingException */ - public void testGetClosest() throws UnsupportedEncodingException { + public void testGetClosest() { addRows(this.hmemcache); - Text closestToEmpty = this.hmemcache.getNextRow(HConstants.EMPTY_TEXT); - assertEquals(closestToEmpty, getRowName(0)); + byte [] closestToEmpty = this.hmemcache.getNextRow(HConstants.EMPTY_BYTE_ARRAY); + assertTrue(Bytes.equals(closestToEmpty, getRowName(0))); for (int i = 0; i < ROW_COUNT; i++) { - Text nr = this.hmemcache.getNextRow(getRowName(i)); + byte [] nr = this.hmemcache.getNextRow(getRowName(i)); if (i + 1 == ROW_COUNT) { assertEquals(nr, null); } else { - assertEquals(nr, getRowName(i + 1)); + assertTrue(Bytes.equals(nr, getRowName(i + 1))); } } } @@ -237,23 +232,25 @@ public class TestHMemcache extends TestCase { public void testScanner() throws IOException { addRows(this.hmemcache); long timestamp = System.currentTimeMillis(); - Text [] cols = new Text[COLUMNS_COUNT * ROW_COUNT]; + byte [][] cols = new byte[COLUMNS_COUNT * ROW_COUNT][]; for (int i = 0; i < ROW_COUNT; i++) { for (int ii = 0; ii < COLUMNS_COUNT; ii++) { cols[(ii + (i * COLUMNS_COUNT))] = getColumnName(i, ii); } } InternalScanner scanner = - this.hmemcache.getScanner(timestamp, cols, new Text()); + this.hmemcache.getScanner(timestamp, cols, HConstants.EMPTY_START_ROW); HStoreKey key = new HStoreKey(); - TreeMap results = new TreeMap(); + TreeMap results = + new TreeMap(Bytes.BYTES_COMPARATOR); for (int i = 0; scanner.next(key, results); i++) { assertTrue("Row name", - key.toString().startsWith(getRowName(i).toString())); + key.toString().startsWith(Bytes.toString(getRowName(i)))); assertEquals("Count of columns", COLUMNS_COUNT, results.size()); - TreeMap row = new TreeMap(); - for(Map.Entry e: results.entrySet() ) { + TreeMap row = + new TreeMap(Bytes.BYTES_COMPARATOR); + for(Map.Entry e: results.entrySet() ) { row.put(e.getKey(), e.getValue()); } isExpectedRowWithoutTimestamps(i, row); @@ -265,11 +262,11 @@ public class TestHMemcache extends TestCase { /** For HBASE-528 */ public void testGetRowKeyAtOrBefore() { // set up some test data - Text t10 = new Text("010"); - Text t20 = new Text("020"); - Text t30 = new Text("030"); - Text t35 = new Text("035"); - Text t40 = new Text("040"); + byte [] t10 = Bytes.toBytes("010"); + byte [] t20 = Bytes.toBytes("020"); + byte [] t30 = Bytes.toBytes("030"); + byte [] t35 = Bytes.toBytes("035"); + byte [] t40 = Bytes.toBytes("040"); hmemcache.add(getHSKForRow(t10), "t10 bytes".getBytes()); hmemcache.add(getHSKForRow(t20), "t20 bytes".getBytes()); @@ -282,7 +279,7 @@ public class TestHMemcache extends TestCase { // try finding "015" results = new TreeMap(); - Text t15 = new Text("015"); + byte [] t15 = Bytes.toBytes("015"); hmemcache.getRowKeyAtOrBefore(t15, results); assertEquals(t10, results.lastKey().getRow()); @@ -293,18 +290,18 @@ public class TestHMemcache extends TestCase { // try "038", should skip the deleted "035" and give "030" results = new TreeMap(); - Text t38 = new Text("038"); + byte [] t38 = Bytes.toBytes("038"); hmemcache.getRowKeyAtOrBefore(t38, results); assertEquals(t30, results.lastKey().getRow()); // try "050", should get stuff from "040" results = new TreeMap(); - Text t50 = new Text("050"); + byte [] t50 = Bytes.toBytes("050"); hmemcache.getRowKeyAtOrBefore(t50, results); assertEquals(t40, results.lastKey().getRow()); } - private HStoreKey getHSKForRow(Text row) { - return new HStoreKey(row, new Text("test_col:"), HConstants.LATEST_TIMESTAMP); + private HStoreKey getHSKForRow(byte [] row) { + return new HStoreKey(row, Bytes.toBytes("test_col:"), HConstants.LATEST_TIMESTAMP); } } \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/src/test/org/apache/hadoop/hbase/regionserver/TestHRegion.java index e994deb0656..08c73985303 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -27,7 +27,6 @@ import java.util.TreeMap; import org.apache.hadoop.dfs.MiniDFSCluster; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.log4j.Logger; @@ -35,6 +34,7 @@ import org.apache.hadoop.hbase.HStoreKey; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.util.Bytes; /** * Basic stand-alone testing of HRegion. @@ -72,13 +72,13 @@ implements RegionUnavailableListener { private static final int FIRST_ROW = 1; private static final int N_ROWS = 1000000; private static final int NUM_VALS = 1000; - private static final Text CONTENTS_BASIC = new Text("contents:basic"); + private static final byte [] CONTENTS_BASIC = Bytes.toBytes("contents:basic"); private static final String CONTENTSTR = "contentstr"; private static final String ANCHORNUM = "anchor:anchornum-"; private static final String ANCHORSTR = "anchorstr"; - private static final Text CONTENTS_BODY = new Text("contents:body"); - private static final Text CONTENTS_FIRSTCOL = new Text("contents:firstcol"); - private static final Text ANCHOR_SECONDCOL = new Text("anchor:secondcol"); + private static final byte [] CONTENTS_BODY = Bytes.toBytes("contents:body"); + private static final byte [] CONTENTS_FIRSTCOL = Bytes.toBytes("contents:firstcol"); + private static final byte [] ANCHOR_SECONDCOL = Bytes.toBytes("anchor:secondcol"); private MiniDFSCluster cluster = null; private HLog log = null; @@ -126,10 +126,10 @@ implements RegionUnavailableListener { for (int k = FIRST_ROW; k <= NUM_VALS; k++) { BatchUpdate batchUpdate = - new BatchUpdate(new Text("row_" + k), System.currentTimeMillis()); + new BatchUpdate(Bytes.toBytes("row_" + k), System.currentTimeMillis()); batchUpdate.put(CONTENTS_BASIC, (CONTENTSTR + k).getBytes(HConstants.UTF8_ENCODING)); - batchUpdate.put(new Text(ANCHORNUM + k), + batchUpdate.put(Bytes.toBytes(ANCHORNUM + k), (ANCHORSTR + k).getBytes(HConstants.UTF8_ENCODING)); region.commit(batchUpdate); } @@ -149,9 +149,9 @@ implements RegionUnavailableListener { startTime = System.currentTimeMillis(); - Text collabel = null; + byte [] collabel = null; for (int k = FIRST_ROW; k <= NUM_VALS; k++) { - Text rowlabel = new Text("row_" + k); + byte [] rowlabel = Bytes.toBytes("row_" + k); byte [] bodydata = region.get(rowlabel, CONTENTS_BASIC).getValue(); assertNotNull(bodydata); @@ -160,7 +160,7 @@ implements RegionUnavailableListener { assertEquals("Incorrect value for key: (" + rowlabel + "," + CONTENTS_BASIC + "), expected: '" + teststr + "' got: '" + bodystr + "'", bodystr, teststr); - collabel = new Text(ANCHORNUM + k); + collabel = Bytes.toBytes(ANCHORNUM + k); bodydata = region.get(rowlabel, collabel).getValue(); bodystr = new String(bodydata, HConstants.UTF8_ENCODING).trim(); teststr = ANCHORSTR + k; @@ -180,9 +180,9 @@ implements RegionUnavailableListener { boolean exceptionThrown = false; exceptionThrown = false; try { - BatchUpdate batchUpdate = new BatchUpdate(new Text("Some old key")); + BatchUpdate batchUpdate = new BatchUpdate(Bytes.toBytes("Some old key")); String unregisteredColName = "FamilyGroup:FamilyLabel"; - batchUpdate.put(new Text(unregisteredColName), + batchUpdate.put(Bytes.toBytes(unregisteredColName), unregisteredColName.getBytes(HConstants.UTF8_ENCODING)); region.commit(batchUpdate); } catch (IOException e) { @@ -205,11 +205,11 @@ implements RegionUnavailableListener { threads.add(new Thread(Integer.toString(i)) { @Override public void run() { - long [] lockids = new long[lockCount]; + Integer [] lockids = new Integer[lockCount]; // Get locks. for (int i = 0; i < lockCount; i++) { try { - Text rowid = new Text(Integer.toString(i)); + byte [] rowid = Bytes.toBytes(Integer.toString(i)); lockids[i] = r.obtainRowLock(rowid); rowid.equals(r.getRowFromLock(lockids[i])); LOG.debug(getName() + " locked " + rowid.toString()); @@ -222,7 +222,7 @@ implements RegionUnavailableListener { // Abort outstanding locks. for (int i = lockCount - 1; i >= 0; i--) { - r.releaseRowLock(r.getRowFromLock(lockids[i])); + r.releaseRowLock(lockids[i]); LOG.debug(getName() + " unlocked " + i); } LOG.debug(getName() + " released " + @@ -252,7 +252,7 @@ implements RegionUnavailableListener { // Test scanners. Writes contents:firstcol and anchor:secondcol private void scan() throws IOException { - Text cols[] = new Text[] { + byte [] cols[] = { CONTENTS_FIRSTCOL, ANCHOR_SECONDCOL }; @@ -271,7 +271,7 @@ implements RegionUnavailableListener { String kLabel = String.format("%1$03d", k); BatchUpdate batchUpdate = - new BatchUpdate(new Text("row_vals1_" + kLabel), + new BatchUpdate(Bytes.toBytes("row_vals1_" + kLabel), System.currentTimeMillis()); batchUpdate.put(cols[0], vals1[k].getBytes(HConstants.UTF8_ENCODING)); batchUpdate.put(cols[1], vals1[k].getBytes(HConstants.UTF8_ENCODING)); @@ -287,20 +287,21 @@ implements RegionUnavailableListener { startTime = System.currentTimeMillis(); InternalScanner s = - r.getScanner(cols, new Text(), System.currentTimeMillis(), null); + r.getScanner(cols, HConstants.EMPTY_START_ROW, System.currentTimeMillis(), null); int numFetched = 0; try { HStoreKey curKey = new HStoreKey(); - TreeMap curVals = new TreeMap(); + TreeMap curVals = + new TreeMap(Bytes.BYTES_COMPARATOR); int k = 0; while(s.next(curKey, curVals)) { - for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { - Text col = it.next(); + for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { + byte [] col = it.next(); byte [] val = curVals.get(col); int curval = Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim()); for(int j = 0; j < cols.length; j++) { - if(col.compareTo(cols[j]) == 0) { + if (Bytes.compareTo(col, cols[j]) == 0) { assertEquals("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp() + ", Value for " + col + " should be: " + k @@ -334,20 +335,22 @@ implements RegionUnavailableListener { startTime = System.currentTimeMillis(); - s = r.getScanner(cols, new Text(), System.currentTimeMillis(), null); + s = r.getScanner(cols, HConstants.EMPTY_START_ROW, + System.currentTimeMillis(), null); numFetched = 0; try { HStoreKey curKey = new HStoreKey(); - TreeMap curVals = new TreeMap(); + TreeMap curVals = + new TreeMap(Bytes.BYTES_COMPARATOR); int k = 0; while(s.next(curKey, curVals)) { - for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { - Text col = it.next(); + for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { + byte [] col = it.next(); byte [] val = curVals.get(col); int curval = Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim()); for(int j = 0; j < cols.length; j++) { - if(col.compareTo(cols[j]) == 0) { + if (Bytes.compareTo(col, cols[j]) == 0) { assertEquals("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp() + ", Value for " + col + " should be: " + k @@ -376,7 +379,7 @@ implements RegionUnavailableListener { String kLabel = String.format("%1$03d", k); BatchUpdate batchUpdate = - new BatchUpdate(new Text("row_vals1_" + kLabel), + new BatchUpdate(Bytes.toBytes("row_vals1_" + kLabel), System.currentTimeMillis()); batchUpdate.put(cols[0], vals1[k].getBytes(HConstants.UTF8_ENCODING)); batchUpdate.put(cols[1], vals1[k].getBytes(HConstants.UTF8_ENCODING)); @@ -391,20 +394,22 @@ implements RegionUnavailableListener { startTime = System.currentTimeMillis(); - s = r.getScanner(cols, new Text(), System.currentTimeMillis(), null); + s = r.getScanner(cols, HConstants.EMPTY_START_ROW, + System.currentTimeMillis(), null); numFetched = 0; try { HStoreKey curKey = new HStoreKey(); - TreeMap curVals = new TreeMap(); + TreeMap curVals = + new TreeMap(Bytes.BYTES_COMPARATOR); int k = 0; while(s.next(curKey, curVals)) { - for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { - Text col = it.next(); + for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { + byte [] col = it.next(); byte [] val = curVals.get(col); int curval = Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim()); for(int j = 0; j < cols.length; j++) { - if(col.compareTo(cols[j]) == 0) { + if(Bytes.compareTo(col, cols[j]) == 0) { assertEquals("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp() + ", Value for " + col + " should be: " + k @@ -438,20 +443,21 @@ implements RegionUnavailableListener { startTime = System.currentTimeMillis(); - s = r.getScanner(cols, new Text(), System.currentTimeMillis(), null); + s = r.getScanner(cols, HConstants.EMPTY_START_ROW, System.currentTimeMillis(), null); numFetched = 0; try { HStoreKey curKey = new HStoreKey(); - TreeMap curVals = new TreeMap(); + TreeMap curVals = + new TreeMap(Bytes.BYTES_COMPARATOR); int k = 0; while(s.next(curKey, curVals)) { - for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { - Text col = it.next(); + for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { + byte [] col = it.next(); byte [] val = curVals.get(col); int curval = Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim()); for (int j = 0; j < cols.length; j++) { - if (col.compareTo(cols[j]) == 0) { + if (Bytes.compareTo(col, cols[j]) == 0) { assertEquals("Value for " + col + " should be: " + k + ", but was fetched as: " + curval, curval, k); numFetched++; @@ -474,22 +480,23 @@ implements RegionUnavailableListener { startTime = System.currentTimeMillis(); - s = r.getScanner(cols, new Text("row_vals1_500"), + s = r.getScanner(cols, Bytes.toBytes("row_vals1_500"), System.currentTimeMillis(), null); numFetched = 0; try { HStoreKey curKey = new HStoreKey(); - TreeMap curVals = new TreeMap(); + TreeMap curVals = + new TreeMap(Bytes.BYTES_COMPARATOR); int k = 500; while(s.next(curKey, curVals)) { - for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { - Text col = it.next(); + for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { + byte [] col = it.next(); byte [] val = curVals.get(col); int curval = Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim()); for (int j = 0; j < cols.length; j++) { - if (col.compareTo(cols[j]) == 0) { + if (Bytes.compareTo(col, cols[j]) == 0) { assertEquals("Value for " + col + " should be: " + k + ", but was fetched as: " + curval, curval, k); numFetched++; @@ -534,7 +541,7 @@ implements RegionUnavailableListener { // Write to the HRegion BatchUpdate batchUpdate = - new BatchUpdate(new Text("row_" + k), System.currentTimeMillis()); + new BatchUpdate(Bytes.toBytes("row_" + k), System.currentTimeMillis()); batchUpdate.put(CONTENTS_BODY, buf1.toString().getBytes(HConstants.UTF8_ENCODING)); region.commit(batchUpdate); @@ -581,7 +588,7 @@ implements RegionUnavailableListener { // NOTE: This test depends on testBatchWrite succeeding private void splitAndMerge() throws IOException { Path oldRegionPath = r.getRegionDir(); - Text midKey = r.compactStores(); + byte [] midKey = r.compactStores(); assertNotNull(midKey); long startTime = System.currentTimeMillis(); HRegion subregions[] = r.splitRegion(this, midKey); @@ -615,14 +622,14 @@ implements RegionUnavailableListener { /** * {@inheritDoc} */ - public void closing(@SuppressWarnings("unused") final Text regionName) { + public void closing(@SuppressWarnings("unused") final byte [] regionName) { // We don't use this here. It is only for the HRegionServer } /** * {@inheritDoc} */ - public void closed(@SuppressWarnings("unused") final Text regionName) { + public void closed(@SuppressWarnings("unused") final byte [] regionName) { // We don't use this here. It is only for the HRegionServer } @@ -632,39 +639,41 @@ implements RegionUnavailableListener { // First verify the data written by testBasic() - Text[] cols = new Text[] { - new Text(ANCHORNUM + "[0-9]+"), - new Text(CONTENTS_BASIC) + byte [][] cols = { + Bytes.toBytes(ANCHORNUM + "[0-9]+"), + CONTENTS_BASIC }; long startTime = System.currentTimeMillis(); InternalScanner s = - r.getScanner(cols, new Text(), System.currentTimeMillis(), null); + r.getScanner(cols, HConstants.EMPTY_START_ROW, + System.currentTimeMillis(), null); try { int contentsFetched = 0; int anchorFetched = 0; HStoreKey curKey = new HStoreKey(); - TreeMap curVals = new TreeMap(); + TreeMap curVals = + new TreeMap(Bytes.BYTES_COMPARATOR); int k = 0; while(s.next(curKey, curVals)) { - for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { - Text col = it.next(); + for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { + byte [] col = it.next(); byte [] val = curVals.get(col); - String curval = new String(val, HConstants.UTF8_ENCODING).trim(); - - if(col.compareTo(CONTENTS_BASIC) == 0) { + String curval = Bytes.toString(val); + if(Bytes.compareTo(col, CONTENTS_BASIC) == 0) { assertTrue("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp() + ", Value for " + col + " should start with: " + CONTENTSTR + ", but was fetched as: " + curval, curval.startsWith(CONTENTSTR)); contentsFetched++; - } else if(col.toString().startsWith(ANCHORNUM)) { + } else if (Bytes.toString(col).startsWith(ANCHORNUM)) { assertTrue("Error at:" + curKey.getRow() + "/" + curKey.getTimestamp() - + ", Value for " + col + " should start with: " + ANCHORSTR + + ", Value for " + Bytes.toString(col) + + " should start with: " + ANCHORSTR + ", but was fetched as: " + curval, curval.startsWith(ANCHORSTR)); anchorFetched++; @@ -689,28 +698,27 @@ implements RegionUnavailableListener { // Verify testScan data - cols = new Text[] { - CONTENTS_FIRSTCOL, - ANCHOR_SECONDCOL - }; + cols = new byte [][] {CONTENTS_FIRSTCOL, ANCHOR_SECONDCOL}; startTime = System.currentTimeMillis(); - s = r.getScanner(cols, new Text(), System.currentTimeMillis(), null); + s = r.getScanner(cols, HConstants.EMPTY_START_ROW, + System.currentTimeMillis(), null); try { int numFetched = 0; HStoreKey curKey = new HStoreKey(); - TreeMap curVals = new TreeMap(); + TreeMap curVals = + new TreeMap(Bytes.BYTES_COMPARATOR); int k = 0; while(s.next(curKey, curVals)) { - for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { - Text col = it.next(); + for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { + byte [] col = it.next(); byte [] val = curVals.get(col); int curval = Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim()); for (int j = 0; j < cols.length; j++) { - if (col.compareTo(cols[j]) == 0) { + if (Bytes.compareTo(col, cols[j]) == 0) { assertEquals("Value for " + col + " should be: " + k + ", but was fetched as: " + curval, curval, k); numFetched++; @@ -730,56 +738,23 @@ implements RegionUnavailableListener { s.close(); } - // Verify testBatchWrite data - -// if(StaticTestEnvironment.debugging) { -// startTime = System.currentTimeMillis(); -// s = r.getScanner(new Text[] { CONTENTS_BODY }, new Text(), -// System.currentTimeMillis(), null); -// -// try { -// int numFetched = 0; -// HStoreKey curKey = new HStoreKey(); -// TreeMap curVals = new TreeMap(); -// int k = 0; -// while(s.next(curKey, curVals)) { -// for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { -// Text col = it.next(); -// byte [] val = curVals.get(col); -// assertTrue(col.compareTo(CONTENTS_BODY) == 0); -// assertNotNull(val); -// numFetched++; -// } -// curVals.clear(); -// k++; -// } -// assertEquals("Inserted " + N_ROWS + " values, but fetched " + numFetched, N_ROWS, numFetched); -// -// LOG.info("Scanned " + N_ROWS -// + " rows from disk. Elapsed time: " -// + ((System.currentTimeMillis() - startTime) / 1000.0)); -// -// } finally { -// s.close(); -// } -// } - // Test a scanner which only specifies the column family name - cols = new Text[] { - new Text("anchor:") + cols = new byte [][] { + Bytes.toBytes("anchor:") }; startTime = System.currentTimeMillis(); - s = r.getScanner(cols, new Text(), System.currentTimeMillis(), null); + s = r.getScanner(cols, HConstants.EMPTY_START_ROW, System.currentTimeMillis(), null); try { int fetched = 0; HStoreKey curKey = new HStoreKey(); - TreeMap curVals = new TreeMap(); + TreeMap curVals = + new TreeMap(Bytes.BYTES_COMPARATOR); while(s.next(curKey, curVals)) { - for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { + for(Iterator it = curVals.keySet().iterator(); it.hasNext(); ) { it.next(); fetched++; } diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java b/src/test/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java new file mode 100644 index 00000000000..fcb22fb774d --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java @@ -0,0 +1,37 @@ +/** + * 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.regionserver; + +import org.apache.hadoop.hbase.HBaseTestCase; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.util.Bytes; + +public class TestHRegionInfo extends HBaseTestCase { + public void testCreateHRegionInfoName() throws Exception { + String tableName = "tablename"; + final byte [] tn = Bytes.toBytes(tableName); + String startKey = "startkey"; + final byte [] sk = Bytes.toBytes(startKey); + String id = "id"; + byte [] name = HRegionInfo.createRegionName(tn, sk, id); + String nameStr = Bytes.toString(name); + assertEquals(nameStr, tableName + "," + startKey + "," + id); + } +} diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java b/src/test/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java index 9af11f7b00f..f8f08df081c 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java @@ -26,9 +26,10 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.dfs.MiniDFSCluster; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.JenkinsHash; 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.hbase.HBaseTestCase; @@ -84,9 +85,9 @@ public class TestHStoreFile extends HBaseTestCase { try { for (char d = FIRST_CHAR; d <= LAST_CHAR; d++) { byte[] b = new byte[] {(byte)d}; - Text t = new Text(new String(b, HConstants.UTF8_ENCODING)); + byte [] t = Bytes.toBytes(new String(b, HConstants.UTF8_ENCODING)); writer.append(new HStoreKey(t, t, System.currentTimeMillis()), - new ImmutableBytesWritable(t.getBytes())); + new ImmutableBytesWritable(t)); } } finally { writer.close(); @@ -106,9 +107,9 @@ public class TestHStoreFile extends HBaseTestCase { for (char d = FIRST_CHAR; d <= LAST_CHAR; d++) { for (char e = FIRST_CHAR; e <= LAST_CHAR; e++) { byte[] b = new byte[] { (byte) d, (byte) e }; - Text t = new Text(new String(b, HConstants.UTF8_ENCODING)); + byte [] t = Bytes.toBytes(new String(b, HConstants.UTF8_ENCODING)); writer.append(new HStoreKey(t, t, System.currentTimeMillis()), - new ImmutableBytesWritable(t.getBytes())); + new ImmutableBytesWritable(t)); } } } finally { @@ -124,8 +125,9 @@ 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.fs, this.dir, getName(), - new Text("colfamily"), 1234567890L, null); + HStoreFile hsf = new HStoreFile(this.conf, this.fs, this.dir, + JenkinsHash.hash(Bytes.toBytes(getName())), + Bytes.toBytes("colfamily"), 1234567890L, null); MapFile.Writer writer = hsf.getWriter(this.fs, SequenceFile.CompressionType.NONE, null); writeStoreFile(writer); @@ -136,14 +138,15 @@ public class TestHStoreFile extends HBaseTestCase { HStoreKey midkey = new HStoreKey(((HStoreKey)reader.midKey()).getRow()); HStoreKey hsk = new HStoreKey(); reader.finalKey(hsk); - Text finalKey = hsk.getRow(); + byte [] finalKey = hsk.getRow(); // Make a reference for the bottom half of the just written file. HStoreFile.Reference reference = new HStoreFile.Reference(hsf.getEncodedRegionName(), hsf.getFileId(), midkey, HStoreFile.Range.top); HStoreFile refHsf = new HStoreFile(this.conf, this.fs, - new Path(DIR, getName()), getName() + "_reference", hsf.getColFamily(), - 456, reference); + new Path(DIR, getName()), + JenkinsHash.hash(Bytes.toBytes(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. refHsf.writeReferenceFiles(this.fs); @@ -165,11 +168,11 @@ public class TestHStoreFile extends HBaseTestCase { boolean first = true; while(halfReader.next(key, value)) { if (first) { - assertEquals(key.getRow().toString(), midkey.getRow().toString()); + assertTrue(Bytes.equals(key.getRow(), midkey.getRow())); first = false; } } - assertEquals(key.getRow().toString(), finalKey.toString()); + assertTrue(Bytes.equals(key.getRow(), finalKey)); } /** @@ -194,7 +197,7 @@ public class TestHStoreFile extends HBaseTestCase { // I know keys are a-z. Let the midkey we want to use be 'd'. See if // HalfMapFiles work even if size of file is < than default MapFile // interval. - checkHalfMapFile(p, new HStoreKey(new Text("d"))); + checkHalfMapFile(p, new HStoreKey("d")); } private WritableComparable getMidkey(final Path p) throws IOException { @@ -245,21 +248,20 @@ public class TestHStoreFile extends HBaseTestCase { assertTrue(key.compareTo(midkey) >= 0); if (first) { first = false; - assertEquals(((HStoreKey)midkey).getRow().toString(), - key.getRow().toString()); + assertTrue(Bytes.equals(((HStoreKey)midkey).getRow(), + key.getRow())); LOG.info("First in top: " + key.toString()); } } LOG.info("Last in top: " + key.toString()); top.getClosest(midkey, value); // Assert value is same as key. - assertEquals(new String(value.get(), HConstants.UTF8_ENCODING), - ((HStoreKey) midkey).getRow().toString()); + assertTrue(Bytes.equals(value.get(), ((HStoreKey) midkey).getRow())); // Next test using a midkey that does not exist in the file. // First, do a key that is < than first key. Ensure splits behave // properly. - WritableComparable badkey = new HStoreKey(new Text(" ")); + WritableComparable badkey = new HStoreKey(" "); bottom = new HStoreFile.HalfMapFileReader(this.fs, p.toString(), this.conf, HStoreFile.Range.bottom, badkey); // When badkey is < than the bottom, should return no values. @@ -273,20 +275,20 @@ public class TestHStoreFile extends HBaseTestCase { if (first) { first = false; LOG.info("First top when key < bottom: " + key.toString()); - String tmp = key.getRow().toString(); + String tmp = Bytes.toString(key.getRow()); for (int i = 0; i < tmp.length(); i++) { assertTrue(tmp.charAt(i) == 'a'); } } } LOG.info("Last top when key < bottom: " + key.toString()); - String tmp = key.getRow().toString(); + String tmp = Bytes.toString(key.getRow()); for (int i = 0; i < tmp.length(); i++) { assertTrue(tmp.charAt(i) == 'z'); } // Test when badkey is > than last key in file ('||' > 'zz'). - badkey = new HStoreKey(new Text("|||")); + badkey = new HStoreKey("|||"); bottom = new HStoreFile.HalfMapFileReader(this.fs, p.toString(), this.conf, HStoreFile.Range.bottom, badkey); first = true; @@ -294,14 +296,14 @@ public class TestHStoreFile extends HBaseTestCase { if (first) { first = false; LOG.info("First bottom when key > top: " + key.toString()); - tmp = key.getRow().toString(); + tmp = Bytes.toString(key.getRow()); for (int i = 0; i < tmp.length(); i++) { assertTrue(tmp.charAt(i) == 'a'); } } } LOG.info("Last bottom when key > top: " + key.toString()); - tmp = key.getRow().toString(); + tmp = Bytes.toString(key.getRow()); for (int i = 0; i < tmp.length(); i++) { assertTrue(tmp.charAt(i) == 'z'); } @@ -338,7 +340,7 @@ public class TestHStoreFile extends HBaseTestCase { // Test using a midkey that does not exist in the file. // First, do a key that is < than first key. Ensure splits behave // properly. - HStoreKey midkey = new HStoreKey(new Text(" ")); + HStoreKey midkey = new HStoreKey(" "); bottom = new HStoreFile.HalfMapFileReader(this.fs, p.toString(), this.conf, HStoreFile.Range.bottom, midkey); // When midkey is < than the bottom, should return no values. @@ -352,14 +354,14 @@ public class TestHStoreFile extends HBaseTestCase { if (first) { first = false; LOG.info("First top when key < bottom: " + key.toString()); - assertEquals("aa", key.getRow().toString()); + assertEquals("aa", Bytes.toString(key.getRow())); } } LOG.info("Last top when key < bottom: " + key.toString()); - assertEquals("zz", key.getRow().toString()); + assertEquals("zz", Bytes.toString(key.getRow())); // Test when midkey is > than last key in file ('||' > 'zz'). - midkey = new HStoreKey(new Text("|||")); + midkey = new HStoreKey("|||"); bottom = new HStoreFile.HalfMapFileReader(this.fs, p.toString(), this.conf, HStoreFile.Range.bottom, midkey); first = true; @@ -367,11 +369,11 @@ public class TestHStoreFile extends HBaseTestCase { if (first) { first = false; LOG.info("First bottom when key > top: " + key.toString()); - assertEquals("aa", key.getRow().toString()); + assertEquals("aa", Bytes.toString(key.getRow())); } } LOG.info("Last bottom when key > top: " + key.toString()); - assertEquals("zz", key.getRow().toString()); + assertEquals("zz", Bytes.toString(key.getRow())); // Now look at top. Should not return any values. top = new HStoreFile.HalfMapFileReader(this.fs, p.toString(), this.conf, HStoreFile.Range.top, midkey); @@ -389,4 +391,4 @@ public class TestHStoreFile extends HBaseTestCase { this.fs.delete(p); } } -} +} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java b/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java index 56b38ec9653..8e5f1a20b3c 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.util.Bytes; /** * Test log deletion as logs are rolled. @@ -62,7 +63,7 @@ public class TestLogRolling extends HBaseClusterTestCase { while (v.length() < 1000) { v.append(className); } - value = v.toString().getBytes(HConstants.UTF8_ENCODING); + value = Bytes.toBytes(v.toString()); } catch (Exception e) { LOG.fatal("error in constructor", e); @@ -107,10 +108,10 @@ public class TestLogRolling extends HBaseClusterTestCase { // Create the test table and open it HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString())); + desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY)); HBaseAdmin admin = new HBaseAdmin(conf); admin.createTable(desc); - HTable table = new HTable(conf, new Text(tableName)); + HTable table = new HTable(conf, tableName); for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls BatchUpdate b = @@ -144,7 +145,7 @@ public class TestLogRolling extends HBaseClusterTestCase { // flush all regions List regions = - new ArrayList(server.getOnlineRegions().values()); + new ArrayList(server.getOnlineRegions()); for (HRegion r: regions) { r.flushcache(); } diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestRegionServerExit.java b/src/test/org/apache/hadoop/hbase/regionserver/TestRegionServerExit.java index b2e711b8558..668b7ed2555 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestRegionServerExit.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestRegionServerExit.java @@ -22,24 +22,20 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.util.Collection; import java.util.List; -import java.util.TreeMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.io.Text; - +import org.apache.hadoop.hbase.HBaseClusterTestCase; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.LocalHBaseCluster; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Scanner; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.RowResult; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.Scanner; - -import org.apache.hadoop.hbase.HBaseClusterTestCase; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HStoreKey; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.LocalHBaseCluster; +import org.apache.hadoop.hbase.util.Bytes; /** * Tests region server failover when a region server exits both cleanly and @@ -67,7 +63,7 @@ public class TestRegionServerExit extends HBaseClusterTestCase { new HTable(conf, HConstants.META_TABLE_NAME); // Create table and add a row. final String tableName = getName(); - Text row = createTableAndAddRow(tableName); + byte [] row = createTableAndAddRow(tableName); // Start up a new region server to take over serving of root and meta // after we shut down the current meta/root host. this.cluster.startRegionServer(); @@ -88,7 +84,7 @@ public class TestRegionServerExit extends HBaseClusterTestCase { new HTable(this.conf, HConstants.META_TABLE_NAME); // Create table and add a row. final String tableName = getName(); - Text row = createTableAndAddRow(tableName); + byte [] row = createTableAndAddRow(tableName); // Start up a new region server to take over serving of root and meta // after we shut down the current meta/root host. this.cluster.startRegionServer(); @@ -100,17 +96,17 @@ public class TestRegionServerExit extends HBaseClusterTestCase { threadDumpingJoin(t); } - private Text createTableAndAddRow(final String tableName) throws IOException { + private byte [] createTableAndAddRow(final String tableName) + throws IOException { HTableDescriptor desc = new HTableDescriptor(tableName); - desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString())); + desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY)); HBaseAdmin admin = new HBaseAdmin(conf); admin.createTable(desc); // put some values in the table - this.table = new HTable(conf, new Text(tableName)); - final Text row = new Text("row1"); + this.table = new HTable(conf, tableName); + byte [] row = Bytes.toBytes("row1"); BatchUpdate b = new BatchUpdate(row); - b.put(HConstants.COLUMN_FAMILY, - tableName.getBytes(HConstants.UTF8_ENCODING)); + b.put(HConstants.COLUMN_FAMILY, Bytes.toBytes(tableName)); table.commit(b); return row; } @@ -129,9 +125,10 @@ public class TestRegionServerExit extends HBaseClusterTestCase { int server = -1; for (int i = 0; i < regionThreads.size() && server == -1; i++) { HRegionServer s = regionThreads.get(i).getRegionServer(); - Collection regions = s.getOnlineRegions().values(); + Collection regions = s.getOnlineRegions(); for (HRegion r : regions) { - if (r.getTableDesc().getName().equals(HConstants.META_TABLE_NAME)) { + if (Bytes.equals(r.getTableDesc().getName(), + HConstants.META_TABLE_NAME)) { server = i; } } @@ -159,7 +156,7 @@ public class TestRegionServerExit extends HBaseClusterTestCase { * @return Verification thread. Caller needs to calls start on it. */ private Thread startVerificationThread(final String tableName, - final Text row) { + final byte [] row) { Runnable runnable = new Runnable() { public void run() { try { @@ -167,7 +164,8 @@ public class TestRegionServerExit extends HBaseClusterTestCase { // meta server comes back up. HTable t = new HTable(conf, HConstants.META_TABLE_NAME); Scanner s = - t.getScanner(HConstants.COLUMN_FAMILY_ARRAY, new Text()); + t.getScanner(HConstants.COLUMN_FAMILY_ARRAY, + HConstants.EMPTY_START_ROW); s.close(); } catch (IOException e) { @@ -179,17 +177,15 @@ public class TestRegionServerExit extends HBaseClusterTestCase { // Verify that the client can find the data after the region has moved // to a different server scanner = - table.getScanner(HConstants.COLUMN_FAMILY_ARRAY, new Text()); + table.getScanner(HConstants.COLUMN_FAMILY_ARRAY, + HConstants.EMPTY_START_ROW); LOG.info("Obtained scanner " + scanner); - HStoreKey key = new HStoreKey(); - TreeMap results = new TreeMap(); for (RowResult r : scanner) { - assertTrue(r.getRow().equals(row)); + assertTrue(Bytes.equals(r.getRow(), row)); assertEquals(1, r.size()); byte[] bytes = r.get(HConstants.COLUMN_FAMILY).getValue(); assertNotNull(bytes); - assertTrue(tableName.equals(new String(bytes, - HConstants.UTF8_ENCODING))); + assertTrue(tableName.equals(Bytes.toString(bytes))); } LOG.info("Success!"); } catch (Exception e) { diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java b/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java index b7839182011..27cd13aa04c 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.util.TreeMap; import org.apache.hadoop.dfs.MiniDFSCluster; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.io.Text; import org.apache.hadoop.hbase.HBaseTestCase; @@ -39,19 +40,19 @@ import org.apache.hadoop.hbase.io.BatchUpdate; * Test of a long-lived scanner validating as we go. */ public class TestScanner extends HBaseTestCase { - private static final Text FIRST_ROW = new Text(); - private static final Text[] COLS = { + private static final byte [] FIRST_ROW = HConstants.EMPTY_START_ROW; + private static final byte [][] COLS = { HConstants.COLUMN_FAMILY }; - private static final Text[] EXPLICIT_COLS = { + private static final byte [][] EXPLICIT_COLS = { HConstants.COL_REGIONINFO, HConstants.COL_SERVER, HConstants.COL_STARTCODE }; - private static final Text ROW_KEY = - new Text(HRegionInfo.rootRegionInfo.getRegionName()); - private static final HRegionInfo REGION_INFO = HRegionInfo.rootRegionInfo; + private static final byte [] ROW_KEY = + HRegionInfo.ROOT_REGIONINFO.getRegionName(); + private static final HRegionInfo REGION_INFO = HRegionInfo.ROOT_REGIONINFO; private static final long START_CODE = Long.MAX_VALUE; @@ -76,9 +77,9 @@ public class TestScanner extends HBaseTestCase { (HRegionInfo) Writables.getWritable(regionBytes, new HRegionInfo()); 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.getStartKey().length); + assertEquals(0, info.getEndKey().length); + assertEquals(0, Bytes.compareTo(info.getRegionName(), REGION_INFO.getRegionName())); assertEquals(0, info.getTableDesc().compareTo(REGION_INFO.getTableDesc())); } @@ -87,10 +88,11 @@ public class TestScanner extends HBaseTestCase { throws IOException { InternalScanner scanner = null; - TreeMap results = new TreeMap(); + TreeMap results = + new TreeMap(Bytes.BYTES_COMPARATOR); HStoreKey key = new HStoreKey(); - Text[][] scanColumns = { + byte [][][] scanColumns = { COLS, EXPLICIT_COLS }; @@ -100,7 +102,7 @@ public class TestScanner extends HBaseTestCase { scanner = r.getScanner(scanColumns[i], FIRST_ROW, System.currentTimeMillis(), null); - while(scanner.next(key, results)) { + while (scanner.next(key, results)) { assertTrue(results.containsKey(HConstants.COL_REGIONINFO)); byte [] val = results.get(HConstants.COL_REGIONINFO); validateRegionInfo(val); @@ -109,7 +111,7 @@ public class TestScanner extends HBaseTestCase { val = results.get(HConstants.COL_STARTCODE); assertNotNull(val); assertFalse(val.length == 0); - long startCode = Writables.bytesToLong(val); + long startCode = Bytes.toLong(val); assertEquals(START_CODE, startCode); } @@ -118,7 +120,7 @@ public class TestScanner extends HBaseTestCase { val = results.get(HConstants.COL_SERVER); assertNotNull(val); assertFalse(val.length == 0); - String server = Writables.bytesToString(val); + String server = Bytes.toString(val); assertEquals(0, server.compareTo(serverName)); } results.clear(); @@ -154,7 +156,7 @@ public class TestScanner extends HBaseTestCase { ByteArrayOutputStream byteStream = new ByteArrayOutputStream(); DataOutputStream s = new DataOutputStream(byteStream); - HRegionInfo.rootRegionInfo.write(s); + HRegionInfo.ROOT_REGIONINFO.write(s); batchUpdate.put(HConstants.COL_REGIONINFO, byteStream.toByteArray()); region.commit(batchUpdate); @@ -181,11 +183,9 @@ public class TestScanner extends HBaseTestCase { batchUpdate = new BatchUpdate(ROW_KEY, System.currentTimeMillis()); - batchUpdate.put(HConstants.COL_SERVER, - Writables.stringToBytes(address.toString())); + batchUpdate.put(HConstants.COL_SERVER, Bytes.toBytes(address.toString())); - batchUpdate.put(HConstants.COL_STARTCODE, - Writables.longToBytes(START_CODE)); + batchUpdate.put(HConstants.COL_STARTCODE, Bytes.toBytes(START_CODE)); region.commit(batchUpdate); @@ -222,7 +222,7 @@ public class TestScanner extends HBaseTestCase { batchUpdate = new BatchUpdate(ROW_KEY, System.currentTimeMillis()); batchUpdate.put(HConstants.COL_SERVER, - Writables.stringToBytes(address.toString())); + Bytes.toBytes(address.toString())); region.commit(batchUpdate); diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestSplit.java b/src/test/org/apache/hadoop/hbase/regionserver/TestSplit.java index c355f53f3dd..5bd2b4adc9d 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestSplit.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestSplit.java @@ -24,7 +24,6 @@ import java.util.TreeMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.io.Text; import org.apache.log4j.Level; import org.apache.log4j.Logger; import org.apache.hadoop.hbase.HStoreKey; @@ -33,6 +32,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.util.Bytes; /** * {@Link TestHRegion} does a split but this TestCase adds testing of fast @@ -87,7 +87,7 @@ public class TestSplit extends HBaseClusterTestCase { private void basicSplit(final HRegion region) throws Exception { addContent(region, COLFAMILY_NAME3); region.flushcache(); - Text midkey = region.compactStores(); + byte [] midkey = region.compactStores(); assertNotNull(midkey); HRegion [] regions = split(region, midkey); try { @@ -99,10 +99,11 @@ public class TestSplit extends HBaseClusterTestCase { } // Assert can get rows out of new regions. Should be able to get first // row from first region and the midkey from second region. - assertGet(regions[0], COLFAMILY_NAME3, new Text(START_KEY)); + assertGet(regions[0], COLFAMILY_NAME3, Bytes.toBytes(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(START_KEY)); + assertScan(regions[0], COLFAMILY_NAME3, + Bytes.toBytes(START_KEY)); assertScan(regions[1], COLFAMILY_NAME3, midkey); // Now prove can't split regions that have references. for (int i = 0; i < regions.length; i++) { @@ -116,7 +117,7 @@ public class TestSplit extends HBaseClusterTestCase { regions[i].flushcache(); } - Text[] midkeys = new Text[regions.length]; + byte [][] midkeys = new byte [regions.length][]; // To make regions splitable force compaction. for (int i = 0; i < regions.length; i++) { midkeys[i] = regions[i].compactStores(); @@ -130,7 +131,7 @@ public class TestSplit extends HBaseClusterTestCase { if (midkeys[i] != null) { rs = split(regions[i], midkeys[i]); for (int j = 0; j < rs.length; j++) { - sortedMap.put(rs[j].getRegionName().toString(), + sortedMap.put(Bytes.toString(rs[j].getRegionName()), openClosedRegion(rs[j])); } } @@ -139,10 +140,9 @@ public class TestSplit extends HBaseClusterTestCase { // 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); + byte[] b = Bytes.toBytes(START_KEY); for (HRegion r : sortedMap.values()) { - assertGet(r, COLFAMILY_NAME3, new Text(new String(b, - HConstants.UTF8_ENCODING))); + assertGet(r, COLFAMILY_NAME3, b); b[0] += interval; } } finally { @@ -156,15 +156,14 @@ public class TestSplit extends HBaseClusterTestCase { } } - private void assertGet(final HRegion r, final String family, final Text k) + private void assertGet(final HRegion r, final byte [] family, final byte [] k) throws IOException { // Now I have k, get values out and assert they are as expected. - Cell[] results = r.get(k, new Text(family), - Integer.MAX_VALUE); + Cell[] results = r.get(k, family, Integer.MAX_VALUE); for (int j = 0; j < results.length; j++) { - Text tmp = new Text(results[j].getValue()); + byte [] tmp = results[j].getValue(); // Row should be equal to value every time. - assertEquals(k.toString(), tmp.toString()); + assertTrue(Bytes.equals(k, tmp)); } } @@ -175,23 +174,24 @@ public class TestSplit extends HBaseClusterTestCase { * @param firstValue * @throws IOException */ - private void assertScan(final HRegion r, final String column, - final Text firstValue) + private void assertScan(final HRegion r, final byte [] column, + final byte [] firstValue) throws IOException { - Text [] cols = new Text[] {new Text(column)}; + byte [][] cols = {column}; InternalScanner s = r.getScanner(cols, HConstants.EMPTY_START_ROW, System.currentTimeMillis(), null); try { HStoreKey curKey = new HStoreKey(); - TreeMap curVals = new TreeMap(); + TreeMap curVals = + new TreeMap(Bytes.BYTES_COMPARATOR); boolean first = true; OUTER_LOOP: while(s.next(curKey, curVals)) { - for(Text col: curVals.keySet()) { + for(byte [] col: curVals.keySet()) { byte [] val = curVals.get(col); - Text curval = new Text(val); + byte [] curval = val; if (first) { first = false; - assertTrue(curval.compareTo(firstValue) == 0); + assertTrue(Bytes.compareTo(curval, firstValue) == 0); } else { // Not asserting anything. Might as well break. break OUTER_LOOP; @@ -203,7 +203,7 @@ public class TestSplit extends HBaseClusterTestCase { } } - private HRegion [] split(final HRegion r, final Text midKey) + private HRegion [] split(final HRegion r, final byte [] midKey) throws IOException { // Assert can get mid key from passed region. assertGet(r, COLFAMILY_NAME3, midKey); diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestTimestamp.java b/src/test/org/apache/hadoop/hbase/regionserver/TestTimestamp.java index ad3dffc60bf..85bda3330ec 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestTimestamp.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestTimestamp.java @@ -20,19 +20,15 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; -import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HStoreKey; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HBaseClusterTestCase; -import org.apache.hadoop.hbase.TimestampTestBase; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseClusterTestCase; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TimestampTestBase; +import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType; +import org.apache.hadoop.hbase.util.Bytes; /** * Tests user specifiable time stamps putting, getting and scanning. Also @@ -44,7 +40,7 @@ public class TestTimestamp extends HBaseClusterTestCase { LogFactory.getLog(TestTimestamp.class.getName()); private static final String COLUMN_NAME = "contents:"; - private static final Text COLUMN = new Text(COLUMN_NAME); + private static final byte [] COLUMN = Bytes.toBytes(COLUMN_NAME); private static final int VERSIONS = 3; /** diff --git a/src/test/org/apache/hadoop/hbase/util/TestMergeTool.java b/src/test/org/apache/hadoop/hbase/util/TestMergeTool.java index d9cb0ca2e3e..51ab0005bfc 100644 --- a/src/test/org/apache/hadoop/hbase/util/TestMergeTool.java +++ b/src/test/org/apache/hadoop/hbase/util/TestMergeTool.java @@ -42,11 +42,11 @@ import org.apache.hadoop.util.ToolRunner; /** Test stand alone merge tool that can merge arbitrary regions */ public class TestMergeTool extends HBaseTestCase { static final Log LOG = LogFactory.getLog(TestMergeTool.class); - protected static final Text COLUMN_NAME = new Text("contents:"); + protected static final byte [] COLUMN_NAME = Bytes.toBytes("contents:"); private final HRegionInfo[] sourceRegions = new HRegionInfo[5]; private final HRegion[] regions = new HRegion[5]; private HTableDescriptor desc; - private Text[][] rows; + private byte [][][] rows; private Path rootdir = null; private MiniDFSCluster dfsCluster = null; private FileSystem fs; @@ -59,45 +59,45 @@ public class TestMergeTool extends HBaseTestCase { // Create table description this.desc = new HTableDescriptor("TestMergeTool"); - this.desc.addFamily(new HColumnDescriptor(COLUMN_NAME.toString())); + this.desc.addFamily(new HColumnDescriptor(COLUMN_NAME)); /* * Create the HRegionInfos for the regions. */ // Region 0 will contain the key range [row_0200,row_0300) - sourceRegions[0] = - new HRegionInfo(this.desc, new Text("row_0200"), new Text("row_0300")); + sourceRegions[0] = new HRegionInfo(this.desc, Bytes.toBytes("row_0200"), + Bytes.toBytes("row_0300")); // Region 1 will contain the key range [row_0250,row_0400) and overlaps // with Region 0 sourceRegions[1] = - new HRegionInfo(this.desc, new Text("row_0250"), new Text("row_0400")); + new HRegionInfo(this.desc, Bytes.toBytes("row_0250"), Bytes.toBytes("row_0400")); // Region 2 will contain the key range [row_0100,row_0200) and is adjacent // to Region 0 or the region resulting from the merge of Regions 0 and 1 sourceRegions[2] = - new HRegionInfo(this.desc, new Text("row_0100"), new Text("row_0200")); + new HRegionInfo(this.desc, Bytes.toBytes("row_0100"), Bytes.toBytes("row_0200")); // Region 3 will contain the key range [row_0500,row_0600) and is not // adjacent to any of Regions 0, 1, 2 or the merged result of any or all // of those regions sourceRegions[3] = - new HRegionInfo(this.desc, new Text("row_0500"), new Text("row_0600")); + new HRegionInfo(this.desc, Bytes.toBytes("row_0500"), Bytes.toBytes("row_0600")); // Region 4 will have empty start and end keys and overlaps all regions. sourceRegions[4] = - new HRegionInfo(this.desc, HConstants.EMPTY_TEXT, HConstants.EMPTY_TEXT); + new HRegionInfo(this.desc, HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY); /* * Now create some row keys */ - this.rows = new Text[5][]; - this.rows[0] = new Text[] { new Text("row_0210"), new Text("row_0280") }; - this.rows[1] = new Text[] { new Text("row_0260"), new Text("row_0350") }; - this.rows[2] = new Text[] { new Text("row_0110"), new Text("row_0175") }; - this.rows[3] = new Text[] { new Text("row_0525"), new Text("row_0560") }; - this.rows[4] = new Text[] { new Text("row_0050"), new Text("row_1000") }; + this.rows = new byte [5][][]; + this.rows[0] = Bytes.toByteArrays(new Text[] { new Text("row_0210"), new Text("row_0280") }); + this.rows[1] = Bytes.toByteArrays(new Text[] { new Text("row_0260"), new Text("row_0350") }); + this.rows[2] = Bytes.toByteArrays(new Text[] { new Text("row_0110"), new Text("row_0175") }); + this.rows[3] = Bytes.toByteArrays(new Text[] { new Text("row_0525"), new Text("row_0560") }); + this.rows[4] = Bytes.toByteArrays(new Text[] { new Text("row_0050"), new Text("row_1000") }); // Start up dfs this.dfsCluster = new MiniDFSCluster(conf, 2, true, (String[])null); @@ -122,21 +122,17 @@ public class TestMergeTool extends HBaseTestCase { * Insert data */ for (int j = 0; j < rows[i].length; j++) { - Text row = rows[i][j]; + byte [] row = rows[i][j]; BatchUpdate b = new BatchUpdate(row); - b.put(COLUMN_NAME, - new ImmutableBytesWritable( - row.getBytes(), 0, row.getLength() - ).get() - ); + b.put(COLUMN_NAME, new ImmutableBytesWritable(row).get()); regions[i].batchUpdate(b); } } // Create root region - HRegion root = HRegion.createHRegion(HRegionInfo.rootRegionInfo, + HRegion root = HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO, this.rootdir, this.conf); // Create meta region - HRegion meta = HRegion.createHRegion(HRegionInfo.firstMetaRegionInfo, + HRegion meta = HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO, this.rootdir, this.conf); // Insert meta into root region HRegion.addRegionToMETA(root, meta); @@ -178,7 +174,7 @@ public class TestMergeTool extends HBaseTestCase { Merge merger = new Merge(this.conf); LOG.info(msg); int errCode = ToolRunner.run(merger, - new String[] {this.desc.getName().toString(), regionName1, regionName2} + new String[] {this.desc.getNameAsString(), regionName1, regionName2} ); assertTrue("'" + msg + "' failed", errCode == 0); HRegionInfo mergedInfo = merger.getMergedHRegionInfo(); @@ -199,8 +195,7 @@ public class TestMergeTool extends HBaseTestCase { for (int j = 0; j < rows[i].length; j++) { byte[] bytes = merged.get(rows[i][j], COLUMN_NAME).getValue(); assertNotNull(rows[i][j].toString(), bytes); - Text value = new Text(bytes); - assertTrue(value.equals(rows[i][j])); + assertTrue(Bytes.equals(bytes, rows[i][j])); } } } @@ -216,8 +211,7 @@ public class TestMergeTool extends HBaseTestCase { for (int j = 0; j < rows[i].length; j++) { byte[] bytes = regions[i].get(rows[i][j], COLUMN_NAME).getValue(); assertNotNull(bytes); - Text value = new Text(bytes); - assertTrue(value.equals(rows[i][j])); + assertTrue(Bytes.equals(bytes, rows[i][j])); } // Close the region and delete the log regions[i].close(); @@ -232,23 +226,23 @@ public class TestMergeTool extends HBaseTestCase { try { // Merge Region 0 and Region 1 HRegion merged = mergeAndVerify("merging regions 0 and 1", - this.sourceRegions[0].getRegionName().toString(), - this.sourceRegions[1].getRegionName().toString(), log, 2); + this.sourceRegions[0].getRegionNameAsString(), + this.sourceRegions[1].getRegionNameAsString(), log, 2); // Merge the result of merging regions 0 and 1 with region 2 merged = mergeAndVerify("merging regions 0+1 and 2", - merged.getRegionInfo().getRegionName().toString(), - this.sourceRegions[2].getRegionName().toString(), log, 3); + merged.getRegionInfo().getRegionNameAsString(), + this.sourceRegions[2].getRegionNameAsString(), log, 3); // Merge the result of merging regions 0, 1 and 2 with region 3 merged = mergeAndVerify("merging regions 0+1+2 and 3", - merged.getRegionInfo().getRegionName().toString(), - this.sourceRegions[3].getRegionName().toString(), log, 4); + merged.getRegionInfo().getRegionNameAsString(), + this.sourceRegions[3].getRegionNameAsString(), log, 4); // Merge the result of merging regions 0, 1, 2 and 3 with region 4 merged = mergeAndVerify("merging regions 0+1+2+3 and 4", - merged.getRegionInfo().getRegionName().toString(), - this.sourceRegions[4].getRegionName().toString(), log, rows.length); + merged.getRegionInfo().getRegionNameAsString(), + this.sourceRegions[4].getRegionNameAsString(), log, rows.length); } finally { log.closeAndDelete(); } diff --git a/src/webapps/master/master.jsp b/src/webapps/master/master.jsp index 98b11498dfb..7f6595171f0 100644 --- a/src/webapps/master/master.jsp +++ b/src/webapps/master/master.jsp @@ -1,6 +1,7 @@ <%@ page contentType="text/html;charset=UTF-8" import="java.util.*" import="org.apache.hadoop.io.Text" + import="org.apache.hadoop.hbase.util.Bytes" import="org.apache.hadoop.hbase.master.HMaster" import="org.apache.hadoop.hbase.HConstants" import="org.apache.hadoop.hbase.master.MetaRegion" @@ -16,7 +17,7 @@ HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER); HBaseConfiguration conf = master.getConfiguration(); HServerAddress rootLocation = master.getRootRegionLocation(); - Map onlineRegions = master.getOnlineMetaRegions(); + Map onlineRegions = master.getOnlineMetaRegions(); Map serverToServerInfos = master.getServersToServerInfo(); int interval = conf.getInt("hbase.regionserver.msginterval", 3000)/1000; @@ -53,10 +54,10 @@ if (rootLocation != null) { %> - + <% if (onlineRegions != null && onlineRegions.size() > 0) { %> - + <% } %>
TableDescription
><%= HConstants.ROOT_TABLE_NAME.toString() %>The -ROOT- table holds references to all .META. regions.
><%= Bytes.toString(HConstants.ROOT_TABLE_NAME) %>The -ROOT- table holds references to all .META. regions.
><%= HConstants.META_TABLE_NAME.toString() %>The .META. table holds references to all User Table regions
><%= Bytes.toString(HConstants.META_TABLE_NAME) %>The .META. table holds references to all User Table regions
@@ -68,7 +69,7 @@ <% for(HTableDescriptor htDesc : tables ) { %> - + <% } %>

<%= tables.length %> table(s) in set.

TableDescription
><%= htDesc.getName() %> <%= htDesc.toString() %>
><%= htDesc.getNameAsString() %> <%= htDesc.toString() %>
diff --git a/src/webapps/master/table.jsp b/src/webapps/master/table.jsp index 5e6d5557cf0..ff552328710 100644 --- a/src/webapps/master/table.jsp +++ b/src/webapps/master/table.jsp @@ -7,12 +7,13 @@ import="org.apache.hadoop.hbase.HServerInfo" import="org.apache.hadoop.hbase.master.HMaster" import="org.apache.hadoop.hbase.master.MetaRegion" + import="org.apache.hadoop.hbase.util.Bytes" import="java.io.IOException" import="java.util.Map" import="org.apache.hadoop.hbase.HConstants"%><% HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER); String tableName = request.getParameter("name"); - HTable table = new HTable(master.getConfiguration(), new Text(tableName)); + HTable table = new HTable(master.getConfiguration(), tableName); Map serverToServerInfos = master.getServersToServerInfo(); String tableHeader = ""; @@ -32,15 +33,15 @@
-<%if(tableName.equals(HConstants.ROOT_TABLE_NAME.toString())) {%> +<%if(tableName.equals(Bytes.toString(HConstants.ROOT_TABLE_NAME))) {%> <%= tableHeader %> <% int infoPort = serverToServerInfos.get(rootLocation.getBindAddress()+":"+rootLocation.getPort()).getInfoPort(); - String url = "http://" + rootLocation.getBindAddress() + ":" + infoPort + "/";%> + String url = "http://" + rootLocation.getHostname() + ":" + infoPort + "/";%>
NameRegion ServerEncoded NameStart KeyEnd Key
<%= tableName %><%= rootLocation.getHostname() %>:<%= rootLocation.getPort() %>--
-<%} else if(tableName.equals(HConstants.META_TABLE_NAME.toString())) { %> +<%} else if(tableName.equals(Bytes.toString(HConstants.META_TABLE_NAME))) { %> <%= tableHeader %> -<% Map onlineRegions = master.getOnlineMetaRegions(); +<% Map onlineRegions = master.getOnlineMetaRegions(); for (MetaRegion meta: onlineRegions.values()) { int infoPort = serverToServerInfos.get(meta.getServer().getBindAddress()+":"+meta.getServer().getPort()).getInfoPort(); String url = "http://" + meta.getServer().getHostname() + ":" + infoPort + "/";%> @@ -59,7 +60,7 @@ System.out.println(hriEntry.getValue().getHostname()+":"+hriEntry.getValue().getPort()); int infoPort = serverToServerInfos.get(hriEntry.getValue().getBindAddress()+":"+hriEntry.getValue().getPort()).getInfoPort(); String url = "http://" + hriEntry.getValue().getHostname().toString() + ":" + infoPort + "/"; %> -<%= hriEntry.getKey().getRegionName()%><%= hriEntry.getValue().getHostname() %>:<%= hriEntry.getValue().getPort() %> +<%= hriEntry.getKey().getRegionNameAsString()%><%= hriEntry.getValue().getHostname() %>:<%= hriEntry.getValue().getPort() %> <%= hriEntry.getKey().getEncodedName()%> <%= hriEntry.getKey().getStartKey()%> <%= hriEntry.getKey().getEndKey()%> <% } %> diff --git a/src/webapps/regionserver/regionserver.jsp b/src/webapps/regionserver/regionserver.jsp index 4f91da8a9ec..fa172872e79 100644 --- a/src/webapps/regionserver/regionserver.jsp +++ b/src/webapps/regionserver/regionserver.jsp @@ -3,12 +3,13 @@ import="org.apache.hadoop.io.Text" import="org.apache.hadoop.hbase.regionserver.HRegionServer" import="org.apache.hadoop.hbase.regionserver.HRegion" + import="org.apache.hadoop.hbase.util.Bytes" import="org.apache.hadoop.hbase.HConstants" import="org.apache.hadoop.hbase.HServerInfo" import="org.apache.hadoop.hbase.HRegionInfo" %><% HRegionServer regionServer = (HRegionServer)getServletContext().getAttribute(HRegionServer.REGIONSERVER); HServerInfo serverInfo = regionServer.getServerInfo(); - Map onlineRegions = regionServer.getOnlineRegions(); + Collection onlineRegions = regionServer.getOnlineRegions(); int interval = regionServer.getConfiguration().getInt("hbase.regionserver.msginterval", 3000)/1000; %> 0) { %> -<% for (HRegion r: onlineRegions.values()) { %> - - +<% for (HRegion r: onlineRegions) { %> + + <% } %>
Region NameEncoded NameStart KeyEnd Key
<%= r.getRegionName().toString() %><%= r.getRegionInfo().getEncodedName() %><%= r.getStartKey().toString() %><%= r.getEndKey().toString() %>
<%= Bytes.toString(r.getRegionName()) %><%= r.getRegionInfo().getEncodedName() %><%= Bytes.toString(r.getStartKey()) %><%= Bytes.toString(r.getEndKey()) %>

Region names are made of the containing table's name, a comma,