:
* @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();
- TreeMapcolumn
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 .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[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[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 extends Writable> 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 extends Writable, ? extends Writable> t) {
- for (Map.Entry extends Writable, ? extends Writable> 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 extends byte [], ? extends V> 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 extends TableMap> 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 extends TableMap> 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 | | | |