.META. and -ROOT-.
*/
- protected HTableDescriptor(final byte [] name, HColumnDescriptor[] families) {
- this.name = name.clone();
- this.nameAsString = Bytes.toString(this.name);
- setMetaFlags(name);
+ protected HTableDescriptor(final TableName name, HColumnDescriptor[] families) {
+ setName(name);
for(HColumnDescriptor descriptor : families) {
this.families.put(descriptor.getName(), descriptor);
}
@@ -273,11 +269,9 @@
* INTERNAL Private constructor used internally creating table descriptors for
* catalog tables, .META. and -ROOT-.
*/
- protected HTableDescriptor(final byte [] name, HColumnDescriptor[] families,
+ protected HTableDescriptor(final TableName name, HColumnDescriptor[] families,
Map[a-zA-Z_0-9.].
* @see HADOOP-1581 HBASE: Un-openable tablename bug
*/
- public HTableDescriptor(final String name) {
- this(Bytes.toBytes(name));
+ public HTableDescriptor(final TableName name) {
+ super();
+ setName(name);
}
/**
* Construct a table descriptor specifying a byte array table name
- * @param name - Table name as a byte array.
- * @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-.].
+ * @param name Table name.
* @see HADOOP-1581 HBASE: Un-openable tablename bug
*/
- public HTableDescriptor(final byte [] name) {
- super();
- setMetaFlags(this.name);
- this.name = this.isMetaRegion()? name: isLegalTableName(name);
- this.nameAsString = Bytes.toString(this.name);
+ @Deprecated
+ public HTableDescriptor(final byte[] name) {
+ this(TableName.valueOf(name));
}
/**
+ * Construct a table descriptor specifying a String table name
+ * @param name Table name.
+ * @see HADOOP-1581 HBASE: Un-openable tablename bug
+ */
+ @Deprecated
+ public HTableDescriptor(final String name) {
+ this(TableName.valueOf(name));
+ }
+
+ /**
* Construct a table descriptor by cloning the descriptor passed as a parameter.
*
* Makes a deep copy of the supplied descriptor.
@@ -334,8 +330,7 @@
*/
public HTableDescriptor(final HTableDescriptor desc) {
super();
- this.name = desc.name.clone();
- this.nameAsString = Bytes.toString(this.name);
+ setName(desc.name);
setMetaFlags(this.name);
for (HColumnDescriptor c: desc.families.values()) {
this.families.put(c.getName(), new HColumnDescriptor(c));
@@ -356,10 +351,9 @@
* Called by constructors.
* @param name
*/
- private void setMetaFlags(final byte [] name) {
- setRootRegion(Bytes.equals(name, HConstants.ROOT_TABLE_NAME));
+ private void setMetaFlags(final TableName name) {
setMetaRegion(isRootRegion() ||
- Bytes.equals(name, HConstants.META_TABLE_NAME));
+ name.equals(TableName.META_TABLE_NAME));
}
/**
@@ -387,10 +381,10 @@
}
/**
- * Checks if this table is either -ROOT- or .META.
+ * Checks if this table is .META.
* region.
*
- * @return true if this is either a -ROOT- or .META.
+ * @return true if this table is .META.
* region
*/
public boolean isMetaRegion() {
@@ -436,59 +430,18 @@
}
/**
- * Checks of the tableName being passed represents either
- * -ROOT- or .META.
+ * Checks of the tableName being passed is a system table
*
- * @return true if a tablesName is either -ROOT-
- * or .META.
+ *
+ * @return true if a tableName is a member of the system
+ * namesapce (aka hbase)
*/
- public static boolean isMetaTable(final byte [] tableName) {
- return Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME) ||
- Bytes.equals(tableName, HConstants.META_TABLE_NAME);
+ public static boolean isSystemTable(final TableName tableName) {
+ return tableName.getNamespaceAsString()
+ .equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR);
}
- // A non-capture group so that this can be embedded.
- public static final String VALID_USER_TABLE_REGEX = "(?:[a-zA-Z_0-9][a-zA-Z_0-9.-]*)";
-
/**
- * Check passed byte buffer, "tableName", is legal user-space table name.
- * @return Returns passed tableName param
- * @throws NullPointerException If passed tableName is null
- * @throws IllegalArgumentException if passed a tableName
- * that is made of other than 'word' characters or underscores: i.e.
- * [a-zA-Z_0-9].
- */
- public static byte [] isLegalTableName(final byte [] tableName) {
- if (tableName == null || tableName.length <= 0) {
- throw new IllegalArgumentException("Name is null or empty");
- }
- if (tableName[0] == '.' || tableName[0] == '-') {
- throw new IllegalArgumentException("Illegal first character <" + tableName[0] +
- "> at 0. User-space table names can only start with 'word " +
- "characters': i.e. [a-zA-Z_0-9]: " + Bytes.toString(tableName));
- }
- if (HConstants.CLUSTER_ID_FILE_NAME.equalsIgnoreCase(Bytes
- .toString(tableName))
- || HConstants.SPLIT_LOGDIR_NAME.equalsIgnoreCase(Bytes
- .toString(tableName))
- || HConstants.VERSION_FILE_NAME.equalsIgnoreCase(Bytes
- .toString(tableName))) {
- throw new IllegalArgumentException(Bytes.toString(tableName)
- + " conflicted with system reserved words");
- }
- for (int i = 0; i < tableName.length; i++) {
- if (Character.isLetterOrDigit(tableName[i]) || tableName[i] == '_' ||
- tableName[i] == '-' || tableName[i] == '.') {
- continue;
- }
- throw new IllegalArgumentException("Illegal character <" + tableName[i] +
- "> at " + i + ". User-space table names can only contain " +
- "'word characters': i.e. [a-zA-Z_0-9-.]: " + Bytes.toString(tableName));
- }
- return tableName;
- }
-
- /**
* Getter for accessing the metadata associated with the key
*
* @param key The key.
@@ -710,12 +663,21 @@
}
/**
+ * Get the name of the table
+ *
+ * @return TableName
+ */
+ public TableName getTableName() {
+ return name;
+ }
+
+ /**
* Get the name of the table as a byte array.
*
* @return name of table
*/
- public byte [] getName() {
- return name;
+ public byte[] getName() {
+ return name.getName();
}
/**
@@ -724,7 +686,7 @@
* @return name of table as a String
*/
public String getNameAsString() {
- return this.nameAsString;
+ return name.getNameAsString();
}
/**
@@ -744,9 +706,14 @@
*
* @param name name of table
*/
+ @Deprecated
public void setName(byte[] name) {
+ setName(TableName.valueOf(name));
+ }
+
+ @Deprecated
+ public void setName(TableName name) {
this.name = name;
- this.nameAsString = Bytes.toString(this.name);
setMetaFlags(this.name);
}
@@ -839,7 +806,7 @@
@Override
public String toString() {
StringBuilder s = new StringBuilder();
- s.append('\'').append(Bytes.toString(name)).append('\'');
+ s.append('\'').append(Bytes.toString(name.getName())).append('\'');
s.append(getValues(true));
for (HColumnDescriptor f : families.values()) {
s.append(", ").append(f);
@@ -853,7 +820,7 @@
*/
public String toStringCustomizedValues() {
StringBuilder s = new StringBuilder();
- s.append('\'').append(Bytes.toString(name)).append('\'');
+ s.append('\'').append(Bytes.toString(name.getName())).append('\'');
s.append(getValues(false));
for(HColumnDescriptor hcd : families.values()) {
s.append(", ").append(hcd.toStringCustomizedValues());
@@ -978,7 +945,7 @@
*/
@Override
public int hashCode() {
- int result = Bytes.hashCode(this.name);
+ int result = this.name.hashCode();
result ^= Byte.valueOf(TABLE_DESCRIPTOR_VERSION).hashCode();
if (this.families != null && this.families.size() > 0) {
for (HColumnDescriptor e: this.families.values()) {
@@ -1002,8 +969,7 @@
if (version < 3)
throw new IOException("versions < 3 are not supported (and never existed!?)");
// version 3+
- name = Bytes.readByteArray(in);
- nameAsString = Bytes.toString(this.name);
+ name = TableName.valueOf(Bytes.readByteArray(in));
setRootRegion(in.readBoolean());
setMetaRegion(in.readBoolean());
values.clear();
@@ -1046,8 +1012,8 @@
@Deprecated
@Override
public void write(DataOutput out) throws IOException {
- out.writeInt(TABLE_DESCRIPTOR_VERSION);
- Bytes.writeByteArray(out, name);
+ out.writeInt(TABLE_DESCRIPTOR_VERSION);
+ Bytes.writeByteArray(out, name.toBytes());
out.writeBoolean(isRootRegion());
out.writeBoolean(isMetaRegion());
out.writeInt(values.size());
@@ -1080,7 +1046,7 @@
*/
@Override
public int compareTo(final HTableDescriptor other) {
- int result = Bytes.compareTo(this.name, other.name);
+ int result = this.name.compareTo(other.name);
if (result == 0) {
result = families.size() - other.families.size();
}
@@ -1350,17 +1316,24 @@
* Returns the {@link Path} object representing the table directory under
* path rootdir
*
+ * Deprecated use FSUtils.getTableDir() instead.
+ *
* @param rootdir qualified path of HBase root directory
* @param tableName name of table
* @return {@link Path} for table
*/
+ @Deprecated
public static Path getTableDir(Path rootdir, final byte [] tableName) {
- return new Path(rootdir, Bytes.toString(tableName));
+ //This is bad I had to mirror code from FSUTils.getTableDir since
+ //there is no module dependency between hbase-client and hbase-server
+ TableName name = TableName.valueOf(tableName);
+ return new Path(rootdir, new Path(HConstants.BASE_NAMESPACE_DIR,
+ new Path(name.getNamespaceAsString(), new Path(name.getQualifierAsString()))));
}
/** Table descriptor for -ROOT- catalog table */
public static final HTableDescriptor ROOT_TABLEDESC = new HTableDescriptor(
- HConstants.ROOT_TABLE_NAME,
+ TableName.ROOT_TABLE_NAME,
new HColumnDescriptor[] {
new HColumnDescriptor(HConstants.CATALOG_FAMILY)
// Ten is arbitrary number. Keep versions to help debugging.
@@ -1373,7 +1346,8 @@
/** Table descriptor for .META. catalog table */
public static final HTableDescriptor META_TABLEDESC = new HTableDescriptor(
- HConstants.META_TABLE_NAME, new HColumnDescriptor[] {
+ TableName.META_TABLE_NAME,
+ new HColumnDescriptor[] {
new HColumnDescriptor(HConstants.CATALOG_FAMILY)
// Ten is arbitrary number. Keep versions to help debugging.
.setMaxVersions(10)
@@ -1395,7 +1369,22 @@
}
}
+ public final static String NAMESPACE_FAMILY_INFO = "info";
+ public final static byte[] NAMESPACE_FAMILY_INFO_BYTES = Bytes.toBytes(NAMESPACE_FAMILY_INFO);
+ public final static byte[] NAMESPACE_COL_DESC_BYTES = Bytes.toBytes("d");
+ /** Table descriptor for namespace table */
+ public static final HTableDescriptor NAMESPACE_TABLEDESC = new HTableDescriptor(
+ TableName.NAMESPACE_TABLE_NAME,
+ new HColumnDescriptor[] {
+ new HColumnDescriptor(NAMESPACE_FAMILY_INFO)
+ // Ten is arbitrary number. Keep versions to help debugging.
+ .setMaxVersions(10)
+ .setInMemory(true)
+ .setBlocksize(8 * 1024)
+ .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+ });
+
@Deprecated
public void setOwner(User owner) {
setOwnerString(owner != null ? owner.getShortName() : null);
@@ -1458,7 +1447,7 @@
*/
public TableSchema convert() {
TableSchema.Builder builder = TableSchema.newBuilder();
- builder.setName(ByteString.copyFrom(getName()));
+ builder.setTableName(ProtobufUtil.toProtoTableName(getTableName()));
for (Map.Entry e: this.values.entrySet()) {
BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder();
aBuilder.setFirst(ByteString.copyFrom(e.getKey().get()));
@@ -1488,7 +1477,9 @@
for (ColumnFamilySchema cfs: list) {
hcds[index++] = HColumnDescriptor.convert(cfs);
}
- HTableDescriptor htd = new HTableDescriptor(ts.getName().toByteArray(), hcds);
+ HTableDescriptor htd = new HTableDescriptor(
+ ProtobufUtil.toTableName(ts.getTableName()),
+ hcds);
for (BytesBytesPair a: ts.getAttributesList()) {
htd.setValue(a.getFirst().toByteArray(), a.getSecond().toByteArray());
}
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/TableExistsException.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/TableExistsException.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/TableExistsException.java (working copy)
@@ -16,6 +16,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.TableName;
/**
* Thrown when a table exists but should not
@@ -37,4 +38,8 @@
public TableExistsException(String s) {
super(s);
}
+
+ public TableExistsException(TableName t) {
+ this(t.getNameAsString());
+ }
}
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotDisabledException.java (working copy)
@@ -20,6 +20,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.util.Bytes;
/**
@@ -48,4 +49,11 @@
public TableNotDisabledException(byte[] tableName) {
this(Bytes.toString(tableName));
}
+
+ /**
+ * @param tableName Name of table that is not disabled
+ */
+ public TableNotDisabledException(TableName tableName) {
+ this(tableName.getNameAsString());
+ }
}
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotEnabledException.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotEnabledException.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotEnabledException.java (working copy)
@@ -20,6 +20,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.util.Bytes;
@@ -46,6 +47,13 @@
/**
* @param tableName Name of table that is not enabled
*/
+ public TableNotEnabledException(TableName tableName) {
+ this(tableName.getNameAsString());
+ }
+
+ /**
+ * @param tableName Name of table that is not enabled
+ */
public TableNotEnabledException(byte[] tableName) {
this(Bytes.toString(tableName));
}
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotFoundException.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotFoundException.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/TableNotFoundException.java (working copy)
@@ -20,6 +20,8 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Bytes;
/** Thrown when a table can not be located */
@InterfaceAudience.Public
@@ -36,4 +38,12 @@
public TableNotFoundException(String s) {
super(s);
}
+
+ public TableNotFoundException(byte[] tableName) {
+ super(Bytes.toString(tableName));
+ }
+
+ public TableNotFoundException(TableName tableName) {
+ super(tableName.getNameAsString());
+ }
}
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/MetaReader.java (working copy)
@@ -21,6 +21,7 @@
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
@@ -72,7 +73,7 @@
* @throws IOException
*/
public static Map fullScan(
- CatalogTracker catalogTracker, final Set disabledTables)
+ CatalogTracker catalogTracker, final Set disabledTables)
throws IOException {
return fullScan(catalogTracker, disabledTables, false);
}
@@ -90,7 +91,7 @@
* @throws IOException
*/
public static Map fullScan(
- CatalogTracker catalogTracker, final Set disabledTables,
+ CatalogTracker catalogTracker, final Set disabledTables,
final boolean excludeOfflinedSplitParents)
throws IOException {
final Map regions =
@@ -102,9 +103,9 @@
Pair region = HRegionInfo.getHRegionInfoAndServerName(r);
HRegionInfo hri = region.getFirst();
if (hri == null) return true;
- if (hri.getTableNameAsString() == null) return true;
+ if (hri.getTableName() == null) return true;
if (disabledTables.contains(
- hri.getTableNameAsString())) return true;
+ hri.getTableName())) return true;
// Are we to include split parents in the list?
if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
regions.put(hri, region.getSecond());
@@ -159,7 +160,7 @@
* @throws IOException
*/
private static HTable getHTable(final CatalogTracker catalogTracker,
- final byte [] tableName)
+ final TableName tableName)
throws IOException {
// Passing the CatalogTracker's connection configuration ensures this
// HTable instance uses the CatalogTracker's connection.
@@ -187,7 +188,7 @@
*/
static HTable getMetaHTable(final CatalogTracker ct)
throws IOException {
- return getHTable(ct, HConstants.META_TABLE_NAME);
+ return getHTable(ct, TableName.META_TABLE_NAME);
}
/**
@@ -274,13 +275,12 @@
* @throws IOException
*/
public static boolean tableExists(CatalogTracker catalogTracker,
- String tableName)
+ final TableName tableName)
throws IOException {
- if (tableName.equals(HTableDescriptor.META_TABLEDESC.getNameAsString())) {
+ if (tableName.equals(HTableDescriptor.META_TABLEDESC.getTableName())) {
// Catalog tables always exist.
return true;
}
- final byte [] tableNameBytes = Bytes.toBytes(tableName);
// Make a version of ResultCollectingVisitor that only collects the first
CollectingVisitor visitor = new CollectingVisitor() {
private HRegionInfo current = null;
@@ -293,7 +293,7 @@
LOG.warn("No serialized HRegionInfo in " + r);
return true;
}
- if (!isInsideTable(this.current, tableNameBytes)) return false;
+ if (!isInsideTable(this.current, tableName)) return false;
// Else call super and add this Result to the collection.
super.visit(r);
// Stop collecting regions from table after we get one.
@@ -306,7 +306,7 @@
this.results.add(this.current);
}
};
- fullScan(catalogTracker, visitor, getTableStartRowForMeta(tableNameBytes));
+ fullScan(catalogTracker, visitor, getTableStartRowForMeta(tableName));
// If visitor has results >= 1 then table exists.
return visitor.getResults().size() >= 1;
}
@@ -319,7 +319,7 @@
* @throws IOException
*/
public static List getTableRegions(CatalogTracker catalogTracker,
- byte [] tableName)
+ TableName tableName)
throws IOException {
return getTableRegions(catalogTracker, tableName, false);
}
@@ -334,7 +334,7 @@
* @throws IOException
*/
public static List getTableRegions(CatalogTracker catalogTracker,
- byte [] tableName, final boolean excludeOfflinedSplitParents)
+ TableName tableName, final boolean excludeOfflinedSplitParents)
throws IOException {
List> result = null;
try {
@@ -361,8 +361,8 @@
* @return True if current tablename is equal to
* tableName
*/
- static boolean isInsideTable(final HRegionInfo current, final byte [] tableName) {
- return Bytes.equals(tableName, current.getTableName());
+ static boolean isInsideTable(final HRegionInfo current, final TableName tableName) {
+ return tableName.equals(current.getTableName());
}
/**
@@ -370,9 +370,9 @@
* @return Place to start Scan in .META. when passed a
* tableName; returns <tableName&rt; <,&rt; <,&rt;
*/
- static byte [] getTableStartRowForMeta(final byte [] tableName) {
- byte [] startRow = new byte[tableName.length + 2];
- System.arraycopy(tableName, 0, startRow, 0, tableName.length);
+ static byte [] getTableStartRowForMeta(TableName tableName) {
+ byte [] startRow = new byte[tableName.getName().length + 2];
+ System.arraycopy(tableName.getName(), 0, startRow, 0, tableName.getName().length);
startRow[startRow.length - 2] = HConstants.DELIMITER;
startRow[startRow.length - 1] = HConstants.DELIMITER;
return startRow;
@@ -387,8 +387,8 @@
* @param tableName bytes of table's name
* @return configured Scan object
*/
- public static Scan getScanForTableName(byte[] tableName) {
- String strName = Bytes.toString(tableName);
+ public static Scan getScanForTableName(TableName tableName) {
+ String strName = tableName.getNameAsString();
// Start key is just the table name with delimiters
byte[] startKey = Bytes.toBytes(strName + ",,");
// Stop key appends the smallest possible char to the table name
@@ -407,9 +407,9 @@
* @throws InterruptedException
*/
public static List>
- getTableRegionsAndLocations(CatalogTracker catalogTracker, String tableName)
+ getTableRegionsAndLocations(CatalogTracker catalogTracker, TableName tableName)
throws IOException, InterruptedException {
- return getTableRegionsAndLocations(catalogTracker, Bytes.toBytes(tableName),
+ return getTableRegionsAndLocations(catalogTracker, tableName,
true);
}
@@ -422,9 +422,9 @@
*/
public static List>
getTableRegionsAndLocations(final CatalogTracker catalogTracker,
- final byte [] tableName, final boolean excludeOfflinedSplitParents)
+ final TableName tableName, final boolean excludeOfflinedSplitParents)
throws IOException, InterruptedException {
- if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
+ if (tableName.equals(TableName.META_TABLE_NAME)) {
// If meta, do a bit of special handling.
ServerName serverName = catalogTracker.getMetaLocation();
List> list =
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java (working copy)
@@ -23,6 +23,9 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
@@ -85,7 +88,7 @@
class AsyncProcess {
private static final Log LOG = LogFactory.getLog(AsyncProcess.class);
protected final HConnection hConnection;
- protected final byte[] tableName;
+ protected final TableName tableName;
protected final ExecutorService pool;
protected final AsyncProcessCallback callback;
protected final BatchErrors errors = new BatchErrors();
@@ -167,7 +170,7 @@
}
}
- public AsyncProcess(HConnection hc, byte[] tableName, ExecutorService pool,
+ public AsyncProcess(HConnection hc, TableName tableName, ExecutorService pool,
AsyncProcessCallback callback, Configuration conf,
RpcRetryingCallerFactory rpcCaller) {
this.hConnection = hc;
@@ -288,7 +291,7 @@
loc = hConnection.locateRegion(this.tableName, row.getRow());
if (loc == null) {
locationException = new IOException("No location found, aborting submit for" +
- " tableName=" + Bytes.toString(tableName) +
+ " tableName=" + tableName +
" rowkey=" + Arrays.toString(row.getRow()));
}
} catch (IOException e) {
@@ -530,7 +533,7 @@
if (toReplay.isEmpty()) {
LOG.warn("Attempt #" + numAttempt + "/" + numTries + " failed for all (" +
initialActions.size() + ") operations on server " + location.getServerName() +
- " NOT resubmitting, tableName=" + Bytes.toString(tableName) + ", location=" + location);
+ " NOT resubmitting, tableName=" + tableName + ", location=" + location);
} else {
submit(initialActions, toReplay, numAttempt, true, errorsByServer);
}
@@ -553,7 +556,7 @@
if (responses == null) {
LOG.info("Attempt #" + numAttempt + "/" + numTries + " failed for all operations" +
" on server " + location.getServerName() + " , trying to resubmit," +
- " tableName=" + Bytes.toString(tableName) + ", location=" + location);
+ " tableName=" + tableName + ", location=" + location);
resubmitAll(initialActions, rsActions, location, numAttempt + 1, null, errorsByServer);
return;
}
@@ -614,7 +617,7 @@
// logs as errors are to be expected wehn region moves, split and so on
LOG.debug("Attempt #" + numAttempt + "/" + numTries + " failed for " + failureCount +
" operations on server " + location.getServerName() + ", resubmitting " +
- toReplay.size() + ", tableName=" + Bytes.toString(tableName) + ", location=" +
+ toReplay.size() + ", tableName=" + tableName + ", location=" +
location + ", last exception was: " + throwable +
" - sleeping " + backOffTime + " ms.");
}
@@ -622,7 +625,7 @@
Thread.sleep(backOffTime);
} catch (InterruptedException e) {
LOG.warn("Not sent: " + toReplay.size() +
- " operations, tableName=" + Bytes.toString(tableName) + ", location=" + location, e);
+ " operations, tableName=" + tableName + ", location=" + location, e);
Thread.interrupted();
return;
}
@@ -631,7 +634,7 @@
} else if (failureCount != 0) {
LOG.warn("Attempt #" + numAttempt + "/" + numTries + " failed for " + failureCount +
" operations on server " + location.getServerName() + " NOT resubmitting." +
- ", tableName=" + Bytes.toString(tableName) + ", location=" + location);
+ ", tableName=" + tableName + ", location=" + location);
}
}
@@ -648,7 +651,7 @@
} catch (InterruptedException e) {
throw new InterruptedIOException("Interrupted." +
" currentNumberOfTask=" + currentNumberOfTask +
- ", tableName=" + Bytes.toString(tableName) + ", tasksDone=" + tasksDone.get());
+ ", tableName=" + tableName + ", tasksDone=" + tasksDone.get());
}
}
}
@@ -666,7 +669,7 @@
lastLog = now;
LOG.info(": Waiting for the global number of running tasks to be equals or less than "
+ max + ", tasksSent=" + tasksSent.get() + ", tasksDone=" + tasksDone.get() +
- ", currentTasksDone=" + currentTasksDone + ", tableName=" + Bytes.toString(tableName));
+ ", currentTasksDone=" + currentTasksDone + ", tableName=" + tableName);
}
waitForNextTaskDone(currentTasksDone);
currentTasksDone = this.tasksDone.get();
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java (working copy)
@@ -26,6 +26,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
@@ -63,7 +64,7 @@
private ScanMetrics scanMetrics = null;
private final long maxScannerResultSize;
private final HConnection connection;
- private final byte[] tableName;
+ private final TableName tableName;
private final int scannerTimeout;
private boolean scanMetricsPublished = false;
private RpcRetryingCaller caller;
@@ -79,7 +80,7 @@
* @throws IOException
*/
public ClientScanner(final Configuration conf, final Scan scan,
- final byte[] tableName) throws IOException {
+ final TableName tableName) throws IOException {
this(conf, scan, tableName, HConnectionManager.getConnection(conf));
}
@@ -94,7 +95,7 @@
* @param connection Connection identifying the cluster
* @throws IOException
*/
- public ClientScanner(final Configuration conf, final Scan scan, final byte[] tableName,
+ public ClientScanner(final Configuration conf, final Scan scan, final TableName tableName,
HConnection connection) throws IOException {
this(conf, scan, tableName, connection, new RpcRetryingCallerFactory(conf));
}
@@ -108,10 +109,10 @@
* @param connection Connection identifying the cluster
* @throws IOException
*/
- public ClientScanner(final Configuration conf, final Scan scan, final byte[] tableName,
+ public ClientScanner(final Configuration conf, final Scan scan, final TableName tableName,
HConnection connection, RpcRetryingCallerFactory rpcFactory) throws IOException {
if (LOG.isTraceEnabled()) {
- LOG.trace("Scan table=" + Bytes.toString(tableName)
+ LOG.trace("Scan table=" + tableName
+ ", startRow=" + Bytes.toStringBinary(scan.getStartRow()));
}
this.scan = scan;
@@ -156,7 +157,7 @@
return this.connection;
}
- protected byte[] getTableName() {
+ protected TableName getTableName() {
return this.tableName;
}
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (working copy)
@@ -22,6 +22,7 @@
import java.io.IOException;
import java.io.InterruptedIOException;
import java.net.SocketTimeoutException;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.LinkedList;
import java.util.List;
@@ -36,6 +37,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -43,6 +45,7 @@
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RegionException;
@@ -83,7 +86,10 @@
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AddColumnRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableRequest;
@@ -248,7 +254,7 @@
* @return True if table exists already.
* @throws IOException
*/
- public boolean tableExists(final String tableName)
+ public boolean tableExists(final TableName tableName)
throws IOException {
boolean b = false;
CatalogTracker ct = getCatalogTracker();
@@ -260,16 +266,16 @@
return b;
}
- /**
- * @param tableName Table to check.
- * @return True if table exists already.
- * @throws IOException
- */
- public boolean tableExists(final byte [] tableName)
+ public boolean tableExists(final byte[] tableName)
throws IOException {
- return tableExists(Bytes.toString(tableName));
+ return tableExists(TableName.valueOf(tableName));
}
+ public boolean tableExists(final String tableName)
+ throws IOException {
+ return tableExists(TableName.valueOf(tableName));
+ }
+
/**
* List all the userspace tables. In other words, scan the META table.
*
@@ -296,7 +302,7 @@
List matched = new LinkedList();
HTableDescriptor[] tables = listTables();
for (HTableDescriptor table : tables) {
- if (pattern.matcher(table.getNameAsString()).matches()) {
+ if (pattern.matcher(table.getTableName().getNameAsString()).matches()) {
matched.add(table);
}
}
@@ -323,11 +329,16 @@
* @throws TableNotFoundException
* @throws IOException if a remote or network exception occurs
*/
- public HTableDescriptor getTableDescriptor(final byte [] tableName)
+ public HTableDescriptor getTableDescriptor(final TableName tableName)
throws TableNotFoundException, IOException {
return this.connection.getHTableDescriptor(tableName);
}
+ public HTableDescriptor getTableDescriptor(final byte[] tableName)
+ throws TableNotFoundException, IOException {
+ return getTableDescriptor(TableName.valueOf(tableName));
+ }
+
private long getPauseTime(int tries) {
int triesCount = tries;
if (triesCount >= HConstants.RETRY_BACKOFF.length) {
@@ -381,7 +392,6 @@
public void createTable(HTableDescriptor desc, byte [] startKey,
byte [] endKey, int numRegions)
throws IOException {
- HTableDescriptor.isLegalTableName(desc.getName());
if(numRegions < 3) {
throw new IllegalArgumentException("Must create at least three regions");
} else if(Bytes.compareTo(startKey, endKey) >= 0) {
@@ -417,11 +427,10 @@
*/
public void createTable(final HTableDescriptor desc, byte [][] splitKeys)
throws IOException {
- HTableDescriptor.isLegalTableName(desc.getName());
try {
createTableAsync(desc, splitKeys);
} catch (SocketTimeoutException ste) {
- LOG.warn("Creating " + desc.getNameAsString() + " took too long", ste);
+ LOG.warn("Creating " + desc.getTableName() + " took too long", ste);
}
int numRegs = splitKeys == null ? 1 : splitKeys.length + 1;
int prevRegCount = 0;
@@ -439,7 +448,7 @@
LOG.warn("No serialized HRegionInfo in " + rowResult);
return true;
}
- if (!(Bytes.equals(info.getTableName(), desc.getName()))) {
+ if (!info.getTableName().equals(desc.getTableName())) {
return false;
}
ServerName serverName = HRegionInfo.getServerName(rowResult);
@@ -451,7 +460,7 @@
return true;
}
};
- MetaScanner.metaScan(conf, connection, visitor, desc.getName());
+ MetaScanner.metaScan(conf, connection, visitor, desc.getTableName());
if (actualRegCount.get() != numRegs) {
if (tries == this.numRetries * this.retryLongerMultiplier - 1) {
throw new RegionOfflineException("Only " + actualRegCount.get() +
@@ -472,7 +481,7 @@
doneWithMetaScan = true;
tries = -1;
}
- } else if (isTableEnabled(desc.getName())) {
+ } else if (isTableEnabled(desc.getTableName())) {
return;
} else {
try { // Sleep
@@ -485,7 +494,7 @@
}
throw new TableNotEnabledException(
"Retries exhausted while still waiting for table: "
- + desc.getNameAsString() + " to be enabled");
+ + desc.getTableName() + " to be enabled");
}
/**
@@ -507,7 +516,9 @@
public void createTableAsync(
final HTableDescriptor desc, final byte [][] splitKeys)
throws IOException {
- HTableDescriptor.isLegalTableName(desc.getName());
+ if(desc.getTableName() == null) {
+ throw new IllegalArgumentException("TableName cannot be null");
+ }
if(splitKeys != null && splitKeys.length > 0) {
Arrays.sort(splitKeys, Bytes.BYTES_COMPARATOR);
// Verify there are no duplicate split keys
@@ -536,17 +547,14 @@
});
}
- /**
- * Deletes a table.
- * Synchronous operation.
- *
- * @param tableName name of table to delete
- * @throws IOException if a remote or network exception occurs
- */
public void deleteTable(final String tableName) throws IOException {
- deleteTable(Bytes.toBytes(tableName));
+ deleteTable(TableName.valueOf(tableName));
}
+ public void deleteTable(final byte[] tableName) throws IOException {
+ deleteTable(TableName.valueOf(tableName));
+ }
+
/**
* Deletes a table.
* Synchronous operation.
@@ -554,8 +562,7 @@
* @param tableName name of table to delete
* @throws IOException if a remote or network exception occurs
*/
- public void deleteTable(final byte [] tableName) throws IOException {
- HTableDescriptor.isLegalTableName(tableName);
+ public void deleteTable(final TableName tableName) throws IOException {
HRegionLocation firstMetaServer = getFirstMetaServerForTable(tableName);
boolean tableExists = true;
@@ -626,11 +633,11 @@
if (tableExists) {
throw new IOException("Retries exhausted, it took too long to wait"+
- " for the table " + Bytes.toString(tableName) + " to be deleted.");
+ " for the table " + tableName + " to be deleted.");
}
// Delete cached information to prevent clients from using old locations
this.connection.clearRegionCache(tableName);
- LOG.info("Deleted " + Bytes.toString(tableName));
+ LOG.info("Deleted " + tableName);
}
/**
@@ -665,9 +672,9 @@
List failed = new LinkedList();
for (HTableDescriptor table : listTables(pattern)) {
try {
- deleteTable(table.getName());
+ deleteTable(table.getTableName());
} catch (IOException ex) {
- LOG.info("Failed to delete table " + table.getNameAsString(), ex);
+ LOG.info("Failed to delete table " + table.getTableName(), ex);
failed.add(table);
}
}
@@ -675,11 +682,6 @@
}
- public void enableTable(final String tableName)
- throws IOException {
- enableTable(Bytes.toBytes(tableName));
- }
-
/**
* Enable a table. May timeout. Use {@link #enableTableAsync(byte[])}
* and {@link #isTableEnabled(byte[])} instead.
@@ -693,16 +695,26 @@
* @see #disableTable(byte[])
* @see #enableTableAsync(byte[])
*/
- public void enableTable(final byte [] tableName)
+ public void enableTable(final TableName tableName)
throws IOException {
enableTableAsync(tableName);
// Wait until all regions are enabled
waitUntilTableIsEnabled(tableName);
- LOG.info("Enabled table " + Bytes.toString(tableName));
+ LOG.info("Enabled table " + tableName);
}
+ public void enableTable(final byte[] tableName)
+ throws IOException {
+ enableTable(TableName.valueOf(tableName));
+ }
+
+ public void enableTable(final String tableName)
+ throws IOException {
+ enableTable(TableName.valueOf(tableName));
+ }
+
/**
* Wait for the table to be enabled and available
* If enabling the table exceeds the retry period, an exception is thrown.
@@ -710,7 +722,7 @@
* @throws IOException if a remote or network exception occurs or
* table is not enabled after the retries period.
*/
- private void waitUntilTableIsEnabled(final byte[] tableName) throws IOException {
+ private void waitUntilTableIsEnabled(final TableName tableName) throws IOException {
boolean enabled = false;
long start = EnvironmentEdgeManager.currentTimeMillis();
for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
@@ -721,7 +733,7 @@
long sleep = getPauseTime(tries);
if (LOG.isDebugEnabled()) {
LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
- "enabled in " + Bytes.toString(tableName));
+ "enabled in " + tableName);
}
try {
Thread.sleep(sleep);
@@ -734,16 +746,11 @@
}
if (!enabled) {
long msec = EnvironmentEdgeManager.currentTimeMillis() - start;
- throw new IOException("Table '" + Bytes.toString(tableName) +
+ throw new IOException("Table '" + tableName +
"' not yet enabled, after " + msec + "ms.");
}
}
- public void enableTableAsync(final String tableName)
- throws IOException {
- enableTableAsync(Bytes.toBytes(tableName));
- }
-
/**
* Brings a table on-line (enables it). Method returns immediately though
* enable of table may take some time to complete, especially if the table
@@ -754,13 +761,13 @@
* @throws IOException
* @since 0.90.0
*/
- public void enableTableAsync(final byte [] tableName)
+ public void enableTableAsync(final TableName tableName)
throws IOException {
- HTableDescriptor.isLegalTableName(tableName);
+ TableName.isLegalFullyQualifiedTableName(tableName.getName());
executeCallable(new MasterAdminCallable(getConnection()) {
@Override
public Void call() throws ServiceException {
- LOG.info("Started enable of " + Bytes.toString(tableName));
+ LOG.info("Started enable of " + tableName);
EnableTableRequest req = RequestConverter.buildEnableTableRequest(tableName);
masterAdmin.enableTable(null,req);
return null;
@@ -768,6 +775,16 @@
});
}
+ public void enableTableAsync(final byte[] tableName)
+ throws IOException {
+ enableTable(TableName.valueOf(tableName));
+ }
+
+ public void enableTableAsync(final String tableName)
+ throws IOException {
+ enableTableAsync(TableName.valueOf(tableName));
+ }
+
/**
* Enable tables matching the passed in pattern and wait on completion.
*
@@ -797,11 +814,11 @@
public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
List failed = new LinkedList();
for (HTableDescriptor table : listTables(pattern)) {
- if (isTableDisabled(table.getName())) {
+ if (isTableDisabled(table.getTableName())) {
try {
- enableTable(table.getName());
+ enableTable(table.getTableName());
} catch (IOException ex) {
- LOG.info("Failed to enable table " + table.getNameAsString(), ex);
+ LOG.info("Failed to enable table " + table.getTableName(), ex);
failed.add(table);
}
}
@@ -809,10 +826,6 @@
return failed.toArray(new HTableDescriptor[failed.size()]);
}
- public void disableTableAsync(final String tableName) throws IOException {
- disableTableAsync(Bytes.toBytes(tableName));
- }
-
/**
* Starts the disable of a table. If it is being served, the master
* will tell the servers to stop serving it. This method returns immediately.
@@ -826,12 +839,12 @@
* @see #isTableEnabled(byte[])
* @since 0.90.0
*/
- public void disableTableAsync(final byte [] tableName) throws IOException {
- HTableDescriptor.isLegalTableName(tableName);
+ public void disableTableAsync(final TableName tableName) throws IOException {
+ TableName.isLegalFullyQualifiedTableName(tableName.getName());
executeCallable(new MasterAdminCallable(getConnection()) {
@Override
public Void call() throws ServiceException {
- LOG.info("Started disable of " + Bytes.toString(tableName));
+ LOG.info("Started disable of " + tableName);
DisableTableRequest req = RequestConverter.buildDisableTableRequest(tableName);
masterAdmin.disableTable(null,req);
return null;
@@ -839,11 +852,14 @@
});
}
- public void disableTable(final String tableName)
- throws IOException {
- disableTable(Bytes.toBytes(tableName));
+ public void disableTableAsync(final byte[] tableName) throws IOException {
+ disableTableAsync(TableName.valueOf(tableName));
}
+ public void disableTableAsync(final String tableName) throws IOException {
+ disableTableAsync(TableName.valueOf(tableName));
+ }
+
/**
* Disable table and wait on completion. May timeout eventually. Use
* {@link #disableTableAsync(byte[])} and {@link #isTableDisabled(String)}
@@ -855,7 +871,7 @@
* TableNotFoundException means the table doesn't exist.
* TableNotEnabledException means the table isn't in enabled state.
*/
- public void disableTable(final byte [] tableName)
+ public void disableTable(final TableName tableName)
throws IOException {
disableTableAsync(tableName);
// Wait until table is disabled
@@ -868,7 +884,7 @@
long sleep = getPauseTime(tries);
if (LOG.isDebugEnabled()) {
LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
- "disabled in " + Bytes.toString(tableName));
+ "disabled in " + tableName);
}
try {
Thread.sleep(sleep);
@@ -881,11 +897,21 @@
}
if (!disabled) {
throw new RegionException("Retries exhausted, it took too long to wait"+
- " for the table " + Bytes.toString(tableName) + " to be disabled.");
+ " for the table " + tableName + " to be disabled.");
}
- LOG.info("Disabled " + Bytes.toString(tableName));
+ LOG.info("Disabled " + tableName);
}
+ public void disableTable(final byte[] tableName)
+ throws IOException {
+ disableTable(TableName.valueOf(tableName));
+ }
+
+ public void disableTable(final String tableName)
+ throws IOException {
+ disableTable(TableName.valueOf(tableName));
+ }
+
/**
* Disable tables matching the passed in pattern and wait on completion.
*
@@ -917,11 +943,11 @@
public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
List failed = new LinkedList();
for (HTableDescriptor table : listTables(pattern)) {
- if (isTableEnabled(table.getName())) {
+ if (isTableEnabled(table.getTableName())) {
try {
- disableTable(table.getName());
+ disableTable(table.getTableName());
} catch (IOException ex) {
- LOG.info("Failed to disable table " + table.getNameAsString(), ex);
+ LOG.info("Failed to disable table " + table.getTableName(), ex);
failed.add(table);
}
}
@@ -934,62 +960,56 @@
* @return true if table is on-line
* @throws IOException if a remote or network exception occurs
*/
- public boolean isTableEnabled(String tableName) throws IOException {
- return isTableEnabled(Bytes.toBytes(tableName));
+ public boolean isTableEnabled(TableName tableName) throws IOException {
+ return connection.isTableEnabled(tableName);
}
- /**
- * @param tableName name of table to check
- * @return true if table is on-line
- * @throws IOException if a remote or network exception occurs
- */
+
public boolean isTableEnabled(byte[] tableName) throws IOException {
- if (!HTableDescriptor.isMetaTable(tableName)) {
- HTableDescriptor.isLegalTableName(tableName);
- }
- return connection.isTableEnabled(tableName);
+ return isTableEnabled(TableName.valueOf(tableName));
}
- /**
- * @param tableName name of table to check
- * @return true if table is off-line
- * @throws IOException if a remote or network exception occurs
- */
- public boolean isTableDisabled(final String tableName) throws IOException {
- return isTableDisabled(Bytes.toBytes(tableName));
+ public boolean isTableEnabled(String tableName) throws IOException {
+ return isTableEnabled(TableName.valueOf(tableName));
}
+
+
/**
* @param tableName name of table to check
* @return true if table is off-line
* @throws IOException if a remote or network exception occurs
*/
- public boolean isTableDisabled(byte[] tableName) throws IOException {
- if (!HTableDescriptor.isMetaTable(tableName)) {
- HTableDescriptor.isLegalTableName(tableName);
- }
+ public boolean isTableDisabled(TableName tableName) throws IOException {
return connection.isTableDisabled(tableName);
}
+ public boolean isTableDisabled(byte[] tableName) throws IOException {
+ return isTableDisabled(TableName.valueOf(tableName));
+ }
+
+ public boolean isTableDisabled(String tableName) throws IOException {
+ return isTableDisabled(TableName.valueOf(tableName));
+ }
+
/**
* @param tableName name of table to check
* @return true if all regions of the table are available
* @throws IOException if a remote or network exception occurs
*/
- public boolean isTableAvailable(byte[] tableName) throws IOException {
+ public boolean isTableAvailable(TableName tableName) throws IOException {
return connection.isTableAvailable(tableName);
}
- /**
- * @param tableName name of table to check
- * @return true if all regions of the table are available
- * @throws IOException if a remote or network exception occurs
- */
+ public boolean isTableAvailable(byte[] tableName) throws IOException {
+ return isTableAvailable(TableName.valueOf(tableName));
+ }
+
public boolean isTableAvailable(String tableName) throws IOException {
- return connection.isTableAvailable(Bytes.toBytes(tableName));
+ return isTableAvailable(TableName.valueOf(tableName));
}
-
+
/**
- * Use this api to check if the table has been created with the specified number of
+ * Use this api to check if the table has been created with the specified number of
* splitkeys which was used while creating the given table.
* Note : If this api is used after a table's region gets splitted, the api may return
* false.
@@ -1000,41 +1020,34 @@
* @throws IOException
* if a remote or network excpetion occurs
*/
- public boolean isTableAvailable(String tableName, byte[][] splitKeys) throws IOException {
- return connection.isTableAvailable(Bytes.toBytes(tableName), splitKeys);
- }
-
- /**
- * Use this api to check if the table has been created with the specified number of
- * splitkeys which was used while creating the given table.
- * Note : If this api is used after a table's region gets splitted, the api may return
- * false.
- * @param tableName
- * name of table to check
- * @param splitKeys
- * keys to check if the table has been created with all split keys
- * @throws IOException
- * if a remote or network excpetion occurs
- */
- public boolean isTableAvailable(byte[] tableName, byte[][] splitKeys) throws IOException {
+ public boolean isTableAvailable(TableName tableName,
+ byte[][] splitKeys) throws IOException {
return connection.isTableAvailable(tableName, splitKeys);
}
+ public boolean isTableAvailable(byte[] tableName,
+ byte[][] splitKeys) throws IOException {
+ return isTableAvailable(TableName.valueOf(tableName), splitKeys);
+ }
+
+ public boolean isTableAvailable(String tableName,
+ byte[][] splitKeys) throws IOException {
+ return isTableAvailable(TableName.valueOf(tableName), splitKeys);
+ }
+
/**
* Get the status of alter command - indicates how many regions have received
* the updated schema Asynchronous operation.
*
- * @param tableName
- * name of the table to get the status of
+ * @param tableName TableName instance
* @return Pair indicating the number of regions updated Pair.getFirst() is the
* regions that are yet to be updated Pair.getSecond() is the total number
* of regions of the table
* @throws IOException
* if a remote or network exception occurs
*/
- public Pair getAlterStatus(final byte[] tableName)
+ public Pair getAlterStatus(final TableName tableName)
throws IOException {
- HTableDescriptor.isLegalTableName(tableName);
return executeCallable(new MasterMonitorCallable>(getConnection()) {
@Override
public Pair call() throws ServiceException {
@@ -1049,6 +1062,23 @@
}
/**
+ * Get the status of alter command - indicates how many regions have received
+ * the updated schema Asynchronous operation.
+ *
+ * @param tableName
+ * name of the table to get the status of
+ * @return Pair indicating the number of regions updated Pair.getFirst() is the
+ * regions that are yet to be updated Pair.getSecond() is the total number
+ * of regions of the table
+ * @throws IOException
+ * if a remote or network exception occurs
+ */
+ public Pair getAlterStatus(final byte[] tableName)
+ throws IOException {
+ return getAlterStatus(TableName.valueOf(tableName));
+ }
+
+ /**
* Add a column to an existing table.
* Asynchronous operation.
*
@@ -1056,9 +1086,23 @@
* @param column column descriptor of column to be added
* @throws IOException if a remote or network exception occurs
*/
+ public void addColumn(final byte[] tableName, HColumnDescriptor column)
+ throws IOException {
+ addColumn(TableName.valueOf(tableName), column);
+ }
+
+
+ /**
+ * Add a column to an existing table.
+ * Asynchronous operation.
+ *
+ * @param tableName name of the table to add column to
+ * @param column column descriptor of column to be added
+ * @throws IOException if a remote or network exception occurs
+ */
public void addColumn(final String tableName, HColumnDescriptor column)
throws IOException {
- addColumn(Bytes.toBytes(tableName), column);
+ addColumn(TableName.valueOf(tableName), column);
}
/**
@@ -1069,7 +1113,7 @@
* @param column column descriptor of column to be added
* @throws IOException if a remote or network exception occurs
*/
- public void addColumn(final byte [] tableName, final HColumnDescriptor column)
+ public void addColumn(final TableName tableName, final HColumnDescriptor column)
throws IOException {
executeCallable(new MasterAdminCallable(getConnection()) {
@Override
@@ -1089,9 +1133,22 @@
* @param columnName name of column to be deleted
* @throws IOException if a remote or network exception occurs
*/
+ public void deleteColumn(final byte[] tableName, final String columnName)
+ throws IOException {
+ deleteColumn(TableName.valueOf(tableName), Bytes.toBytes(columnName));
+ }
+
+ /**
+ * Delete a column from a table.
+ * Asynchronous operation.
+ *
+ * @param tableName name of table
+ * @param columnName name of column to be deleted
+ * @throws IOException if a remote or network exception occurs
+ */
public void deleteColumn(final String tableName, final String columnName)
throws IOException {
- deleteColumn(Bytes.toBytes(tableName), Bytes.toBytes(columnName));
+ deleteColumn(TableName.valueOf(tableName), Bytes.toBytes(columnName));
}
/**
@@ -1102,7 +1159,7 @@
* @param columnName name of column to be deleted
* @throws IOException if a remote or network exception occurs
*/
- public void deleteColumn(final byte [] tableName, final byte [] columnName)
+ public void deleteColumn(final TableName tableName, final byte [] columnName)
throws IOException {
executeCallable(new MasterAdminCallable(getConnection()) {
@Override
@@ -1124,11 +1181,24 @@
*/
public void modifyColumn(final String tableName, HColumnDescriptor descriptor)
throws IOException {
- modifyColumn(Bytes.toBytes(tableName), descriptor);
+ modifyColumn(TableName.valueOf(tableName), descriptor);
}
+ /**
+ * Modify an existing column family on a table.
+ * Asynchronous operation.
+ *
+ * @param tableName name of table
+ * @param descriptor new column descriptor to use
+ * @throws IOException if a remote or network exception occurs
+ */
+ public void modifyColumn(final byte[] tableName, HColumnDescriptor descriptor)
+ throws IOException {
+ modifyColumn(TableName.valueOf(tableName), descriptor);
+ }
+
/**
* Modify an existing column family on a table.
* Asynchronous operation.
@@ -1137,7 +1207,7 @@
* @param descriptor new column descriptor to use
* @throws IOException if a remote or network exception occurs
*/
- public void modifyColumn(final byte [] tableName, final HColumnDescriptor descriptor)
+ public void modifyColumn(final TableName tableName, final HColumnDescriptor descriptor)
throws IOException {
executeCallable(new MasterAdminCallable(getConnection()) {
@Override
@@ -1286,7 +1356,7 @@
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
- public void flush(final byte [] tableNameOrRegionName)
+ public void flush(final byte[] tableNameOrRegionName)
throws IOException, InterruptedException {
CatalogTracker ct = getCatalogTracker();
try {
@@ -1299,7 +1369,8 @@
flush(regionServerPair.getSecond(), regionServerPair.getFirst());
}
} else {
- final String tableName = tableNameString(tableNameOrRegionName, ct);
+ final TableName tableName = checkTableExists(
+ TableName.valueOf(tableNameOrRegionName), ct);
List> pairs =
MetaReader.getTableRegionsAndLocations(ct,
tableName);
@@ -1354,7 +1425,7 @@
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
- public void compact(final byte [] tableNameOrRegionName)
+ public void compact(final byte[] tableNameOrRegionName)
throws IOException, InterruptedException {
compact(tableNameOrRegionName, null, false);
}
@@ -1382,7 +1453,7 @@
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
- public void compact(final byte [] tableNameOrRegionName, final byte[] columnFamily)
+ public void compact(final byte[] tableNameOrRegionName, final byte[] columnFamily)
throws IOException, InterruptedException {
compact(tableNameOrRegionName, columnFamily, false);
}
@@ -1408,7 +1479,7 @@
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
- public void majorCompact(final byte [] tableNameOrRegionName)
+ public void majorCompact(final byte[] tableNameOrRegionName)
throws IOException, InterruptedException {
compact(tableNameOrRegionName, null, true);
}
@@ -1437,7 +1508,7 @@
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
- public void majorCompact(final byte [] tableNameOrRegionName,
+ public void majorCompact(final byte[] tableNameOrRegionName,
final byte[] columnFamily) throws IOException, InterruptedException {
compact(tableNameOrRegionName, columnFamily, true);
}
@@ -1452,7 +1523,7 @@
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
- private void compact(final byte [] tableNameOrRegionName,
+ private void compact(final byte[] tableNameOrRegionName,
final byte[] columnFamily,final boolean major)
throws IOException, InterruptedException {
CatalogTracker ct = getCatalogTracker();
@@ -1466,7 +1537,8 @@
compact(regionServerPair.getSecond(), regionServerPair.getFirst(), major, columnFamily);
}
} else {
- final String tableName = tableNameString(tableNameOrRegionName, ct);
+ final TableName tableName =
+ checkTableExists(TableName.valueOf(tableNameOrRegionName), ct);
List> pairs =
MetaReader.getTableRegionsAndLocations(ct,
tableName);
@@ -1749,7 +1821,7 @@
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
- public void split(final byte [] tableNameOrRegionName)
+ public void split(final byte[] tableNameOrRegionName)
throws IOException, InterruptedException {
split(tableNameOrRegionName, null);
}
@@ -1768,7 +1840,7 @@
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException interrupt exception occurred
*/
- public void split(final byte [] tableNameOrRegionName,
+ public void split(final byte[] tableNameOrRegionName,
final byte [] splitPoint) throws IOException, InterruptedException {
CatalogTracker ct = getCatalogTracker();
try {
@@ -1781,7 +1853,8 @@
split(regionServerPair.getSecond(), regionServerPair.getFirst(), splitPoint);
}
} else {
- final String tableName = tableNameString(tableNameOrRegionName, ct);
+ final TableName tableName =
+ checkTableExists(TableName.valueOf(tableNameOrRegionName), ct);
List> pairs =
MetaReader.getTableRegionsAndLocations(ct,
tableName);
@@ -1817,11 +1890,11 @@
* @param htd modified description of the table
* @throws IOException if a remote or network exception occurs
*/
- public void modifyTable(final byte [] tableName, final HTableDescriptor htd)
+ public void modifyTable(final TableName tableName, final HTableDescriptor htd)
throws IOException {
- if (!Bytes.equals(tableName, htd.getName())) {
- throw new IllegalArgumentException("the specified table name '" + Bytes.toString(tableName) +
- "' doesn't match with the HTD one: " + htd.getNameAsString());
+ if (!tableName.equals(htd.getTableName())) {
+ throw new IllegalArgumentException("the specified table name '" + tableName +
+ "' doesn't match with the HTD one: " + htd.getTableName());
}
executeCallable(new MasterAdminCallable(getConnection()) {
@@ -1834,6 +1907,16 @@
});
}
+ public void modifyTable(final byte[] tableName, final HTableDescriptor htd)
+ throws IOException {
+ modifyTable(TableName.valueOf(tableName), htd);
+ }
+
+ public void modifyTable(final String tableName, final HTableDescriptor htd)
+ throws IOException {
+ modifyTable(TableName.valueOf(tableName), htd);
+ }
+
/**
* @param tableNameOrRegionName Name of a table or name of a region.
* @param ct A {@link CatalogTracker} instance (caller of this method usually has one).
@@ -1875,21 +1958,21 @@
}
/**
- * Convert the table name byte array into a table name string and check if table
- * exists or not.
- * @param tableNameBytes Name of a table.
+ * Check if table exists or not
+ * @param tableName Name of a table.
* @param ct A {@link CatalogTracker} instance (caller of this method usually has one).
- * @return tableName in string form.
+ * @return tableName instance
* @throws IOException if a remote or network exception occurs.
* @throws TableNotFoundException if table does not exist.
*/
- private String tableNameString(final byte[] tableNameBytes, CatalogTracker ct)
+ //TODO rename this method
+ private TableName checkTableExists(
+ final TableName tableName, CatalogTracker ct)
throws IOException {
- String tableNameString = Bytes.toString(tableNameBytes);
- if (!MetaReader.tableExists(ct, tableNameString)) {
- throw new TableNotFoundException(tableNameString);
+ if (!MetaReader.tableExists(ct, tableName)) {
+ throw new TableNotFoundException(tableName);
}
- return tableNameString;
+ return tableName;
}
/**
@@ -1958,9 +2041,9 @@
});
}
- private HRegionLocation getFirstMetaServerForTable(final byte [] tableName)
+ private HRegionLocation getFirstMetaServerForTable(final TableName tableName)
throws IOException {
- return connection.locateRegion(HConstants.META_TABLE_NAME,
+ return connection.locateRegion(TableName.META_TABLE_NAME,
HRegionInfo.createRegionName(tableName, null, HConstants.NINES, false));
}
@@ -1972,6 +2055,127 @@
}
/**
+ * Create a new namespace
+ * @param descriptor descriptor which describes the new namespace
+ * @throws IOException
+ */
+ public void createNamespace(final NamespaceDescriptor descriptor) throws IOException {
+ executeCallable(new MasterAdminCallable(getConnection()) {
+ @Override
+ public Void call() throws Exception {
+ masterAdmin.createNamespace(null,
+ MasterAdminProtos.CreateNamespaceRequest.newBuilder()
+ .setNamespaceDescriptor(ProtobufUtil
+ .toProtoNamespaceDescriptor(descriptor)).build());
+ return null;
+ }
+ });
+ }
+
+ /**
+ * Modify an existing namespace
+ * @param descriptor descriptor which describes the new namespace
+ * @throws IOException
+ */
+ public void modifyNamespace(final NamespaceDescriptor descriptor) throws IOException {
+ executeCallable(new MasterAdminCallable(getConnection()) {
+ @Override
+ public Void call() throws Exception {
+ masterAdmin.modifyNamespace(null,
+ MasterAdminProtos.ModifyNamespaceRequest.newBuilder()
+ .setNamespaceDescriptor(ProtobufUtil
+ .toProtoNamespaceDescriptor(descriptor)).build());
+ return null;
+ }
+ });
+ }
+
+ /**
+ * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
+ * @param name namespace name
+ * @throws IOException
+ */
+ public void deleteNamespace(final String name) throws IOException {
+ executeCallable(new MasterAdminCallable(getConnection()) {
+ @Override
+ public Void call() throws Exception {
+ masterAdmin.deleteNamespace(null,
+ MasterAdminProtos.DeleteNamespaceRequest.newBuilder()
+ .setNamespaceName(name).build());
+ return null;
+ }
+ });
+ }
+
+ /**
+ * Get a namespace descriptor by name
+ * @param name name of namespace descriptor
+ * @return
+ * @throws IOException
+ */
+ public NamespaceDescriptor getNamespaceDescriptor(final String name) throws IOException {
+ return
+ executeCallable(new MasterAdminCallable(getConnection()) {
+ @Override
+ public NamespaceDescriptor call() throws Exception {
+ return ProtobufUtil.toNamespaceDescriptor(
+ masterAdmin.getNamespaceDescriptor(null,
+ MasterAdminProtos.GetNamespaceDescriptorRequest.newBuilder()
+ .setNamespaceName(name).build()).getNamespaceDescriptor());
+ }
+ });
+ }
+
+ /**
+ * List available namespace descriptors
+ * @return
+ * @throws IOException
+ */
+ public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
+ return
+ executeCallable(new MasterAdminCallable(getConnection()) {
+ @Override
+ public NamespaceDescriptor[] call() throws Exception {
+ List list =
+ masterAdmin.listNamespaceDescriptors(null,
+ MasterAdminProtos.ListNamespaceDescriptorsRequest.newBuilder().build())
+ .getNamespaceDescriptorList();
+ NamespaceDescriptor[] res = new NamespaceDescriptor[list.size()];
+ for(int i = 0; i < list.size(); i++) {
+ res[i] = ProtobufUtil.toNamespaceDescriptor(list.get(i));
+ }
+ return res;
+ }
+ });
+ }
+
+ /**
+ * Get list of table descriptors by namespace
+ * @param name namespace name
+ * @return
+ * @throws IOException
+ */
+ public HTableDescriptor[] getTableDescriptorsByNamespace(final String name) throws IOException {
+ return
+ executeCallable(new MasterAdminCallable(getConnection()) {
+ @Override
+ public HTableDescriptor[] call() throws Exception {
+ List list =
+ masterAdmin.getTableDescriptorsByNamespace(null,
+ MasterAdminProtos.GetTableDescriptorsByNamespaceRequest.newBuilder()
+ .setNamespaceName(name).build())
+ .getTableSchemaList();
+ HTableDescriptor[] res = new HTableDescriptor[list.size()];
+ for(int i=0; i < list.size(); i++) {
+
+ res[i] = HTableDescriptor.convert(list.get(i));
+ }
+ return res;
+ }
+ });
+ }
+
+ /**
* Check to see if HBase is running. Throw an exception if not.
* We consider that HBase is running if ZooKeeper and Master are running.
*
@@ -2029,7 +2233,7 @@
* @return Ordered list of {@link HRegionInfo}.
* @throws IOException
*/
- public List getTableRegions(final byte[] tableName)
+ public List getTableRegions(final TableName tableName)
throws IOException {
CatalogTracker ct = getCatalogTracker();
List Regions = null;
@@ -2041,6 +2245,11 @@
return Regions;
}
+ public List getTableRegions(final byte[] tableName)
+ throws IOException {
+ return getTableRegions(TableName.valueOf(tableName));
+ }
+
@Override
public void close() throws IOException {
if (cleanupConnectionOnClose && this.connection != null) {
@@ -2048,18 +2257,34 @@
}
}
- /**
- * Get tableDescriptors
- * @param tableNames List of table names
- * @return HTD[] the tableDescriptor
- * @throws IOException if a remote or network exception occurs
- */
- public HTableDescriptor[] getTableDescriptors(List tableNames)
+ /**
+ * Get tableDescriptors
+ * @param tableNames List of table names
+ * @return HTD[] the tableDescriptor
+ * @throws IOException if a remote or network exception occurs
+ */
+ public HTableDescriptor[] getTableDescriptorsByTableName(List tableNames)
throws IOException {
- return this.connection.getHTableDescriptors(tableNames);
+ return this.connection.getHTableDescriptorsByTableName(tableNames);
}
/**
+ * Get tableDescriptors
+ * @param tableNames List of table names
+ * @return HTD[] the tableDescriptor
+ * @throws IOException if a remote or network exception occurs
+ */
+ public HTableDescriptor[] getTableDescriptors(List names)
+ throws IOException {
+ List tableNames = new ArrayList(names.size());
+ for(String name : names) {
+ tableNames.add(TableName.valueOf(name));
+ }
+ return getTableDescriptorsByTableName(tableNames);
+ }
+
+
+ /**
* Roll the log writer. That is, start writing log messages to a new file.
*
* @param serverName
@@ -2123,7 +2348,7 @@
* @throws InterruptedException
* @return the current compaction state
*/
- public CompactionState getCompactionState(final byte [] tableNameOrRegionName)
+ public CompactionState getCompactionState(final byte[] tableNameOrRegionName)
throws IOException, InterruptedException {
CompactionState state = CompactionState.NONE;
CatalogTracker ct = getCatalogTracker();
@@ -2142,7 +2367,8 @@
return response.getCompactionState();
}
} else {
- final String tableName = tableNameString(tableNameOrRegionName, ct);
+ final TableName tableName =
+ checkTableExists(TableName.valueOf(tableNameOrRegionName), ct);
List> pairs =
MetaReader.getTableRegionsAndLocations(ct, tableName);
for (Pair pair: pairs) {
@@ -2198,38 +2424,56 @@
* a {@link SnapshotCreationException} indicating the duplicate naming.
*
* Snapshot names follow the same naming constraints as tables in HBase. See
- * {@link HTableDescriptor#isLegalTableName(byte[])}.
+ * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
* @param snapshotName name of the snapshot to be created
* @param tableName name of the table for which snapshot is created
* @throws IOException if a remote or network exception occurs
* @throws SnapshotCreationException if snapshot creation failed
* @throws IllegalArgumentException if the snapshot request is formatted incorrectly
*/
- public void snapshot(final String snapshotName, final String tableName) throws IOException,
+ public void snapshot(final String snapshotName,
+ final TableName tableName) throws IOException,
SnapshotCreationException, IllegalArgumentException {
snapshot(snapshotName, tableName, SnapshotDescription.Type.FLUSH);
}
+ public void snapshot(final String snapshotName,
+ final String tableName) throws IOException,
+ SnapshotCreationException, IllegalArgumentException {
+ snapshot(snapshotName, TableName.valueOf(tableName),
+ SnapshotDescription.Type.FLUSH);
+ }
+
/**
- * Create a timestamp consistent snapshot for the given table.
- *
- * Snapshots are considered unique based on the name of the snapshot. Attempts to take a
- * snapshot with the same name (even a different type or with different parameters) will fail with
- * a {@link SnapshotCreationException} indicating the duplicate naming.
- *
- * Snapshot names follow the same naming constraints as tables in HBase. See
- * {@link HTableDescriptor#isLegalTableName(byte[])}.
- * @param snapshotName name of the snapshot to be created
- * @param tableName name of the table for which snapshot is created
- * @throws IOException if a remote or network exception occurs
- * @throws SnapshotCreationException if snapshot creation failed
- * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
- */
- public void snapshot(final byte[] snapshotName, final byte[] tableName) throws IOException,
+ public void snapshot(final String snapshotName,
+ * Create a timestamp consistent snapshot for the given table.
+ final byte[] tableName) throws IOException,
+ *
+ * Snapshots are considered unique based on the name of the snapshot. Attempts to take a
+ * snapshot with the same name (even a different type or with different parameters) will fail with
+ * a {@link SnapshotCreationException} indicating the duplicate naming.
+ *
+ * Snapshot names follow the same naming constraints as tables in HBase. See
+ * {@link HTableDescriptor#isLegalTableName(byte[])}.
+ * @param snapshotName name of the snapshot to be created
+ * @param tableName name of the table for which snapshot is created
+ * @throws IOException if a remote or network exception occurs
+ * @throws SnapshotCreationException if snapshot creation failed
+ * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
+ */
+ public void snapshot(final byte[] snapshotName,
+ final TableName tableName) throws IOException,
SnapshotCreationException, IllegalArgumentException {
- snapshot(Bytes.toString(snapshotName), Bytes.toString(tableName));
+ snapshot(Bytes.toString(snapshotName), tableName, SnapshotDescription.Type.FLUSH);
}
+ public void snapshot(final byte[] snapshotName,
+ final byte[] tableName) throws IOException,
+ SnapshotCreationException, IllegalArgumentException {
+ snapshot(Bytes.toString(snapshotName), TableName.valueOf(tableName),
+ SnapshotDescription.Type.FLUSH);
+ }
+
/**
* Create typed snapshot of the table.
*
@@ -2238,7 +2482,7 @@
* a {@link SnapshotCreationException} indicating the duplicate naming.
*
* Snapshot names follow the same naming constraints as tables in HBase. See
- * {@link HTableDescriptor#isLegalTableName(byte[])}.
+ * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
*
* @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other
* snapshots stored on the cluster
@@ -2248,16 +2492,31 @@
* @throws SnapshotCreationException if snapshot creation failed
* @throws IllegalArgumentException if the snapshot request is formatted incorrectly
*/
- public void snapshot(final String snapshotName, final String tableName,
- SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
+ public void snapshot(final String snapshotName,
+ final TableName tableName,
+ SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
IllegalArgumentException {
SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
- builder.setTable(tableName);
+ builder.setTable(tableName.getNameAsString());
builder.setName(snapshotName);
builder.setType(type);
snapshot(builder.build());
}
+ public void snapshot(final String snapshotName,
+ final String tableName,
+ SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
+ IllegalArgumentException {
+ snapshot(snapshotName, TableName.valueOf(tableName), type);
+ }
+
+ public void snapshot(final String snapshotName,
+ final byte[] tableName,
+ SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
+ IllegalArgumentException {
+ snapshot(snapshotName, TableName.valueOf(tableName), type);
+ }
+
/**
* Take a snapshot and wait for the server to complete that snapshot (blocking).
*
@@ -2270,7 +2529,7 @@
* a {@link SnapshotCreationException} indicating the duplicate naming.
*
* Snapshot names follow the same naming constraints as tables in HBase. See
- * {@link HTableDescriptor#isLegalTableName(byte[])}.
+ * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
*
* You should probably use {@link #snapshot(String, String)} or {@link #snapshot(byte[], byte[])}
* unless you are sure about the type of snapshot that you want to take.
@@ -2406,10 +2665,10 @@
throws IOException, RestoreSnapshotException {
String rollbackSnapshot = snapshotName + "-" + EnvironmentEdgeManager.currentTimeMillis();
- String tableName = null;
+ TableName tableName = null;
for (SnapshotDescription snapshotInfo: listSnapshots()) {
if (snapshotInfo.getName().equals(snapshotName)) {
- tableName = snapshotInfo.getTable();
+ tableName = TableName.valueOf(snapshotInfo.getTable());
break;
}
}
@@ -2453,7 +2712,7 @@
*/
public void cloneSnapshot(final byte[] snapshotName, final byte[] tableName)
throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
- cloneSnapshot(Bytes.toString(snapshotName), Bytes.toString(tableName));
+ cloneSnapshot(Bytes.toString(snapshotName), TableName.valueOf(tableName));
}
/**
@@ -2466,13 +2725,45 @@
* @throws RestoreSnapshotException if snapshot failed to be cloned
* @throws IllegalArgumentException if the specified table has not a valid name
*/
+ public void cloneSnapshot(final byte[] snapshotName, final TableName tableName)
+ throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
+ cloneSnapshot(Bytes.toString(snapshotName), tableName);
+ }
+
+
+
+ /**
+ * Create a new table by cloning the snapshot content.
+ *
+ * @param snapshotName name of the snapshot to be cloned
+ * @param tableName name of the table where the snapshot will be restored
+ * @throws IOException if a remote or network exception occurs
+ * @throws TableExistsException if table to be created already exists
+ * @throws RestoreSnapshotException if snapshot failed to be cloned
+ * @throws IllegalArgumentException if the specified table has not a valid name
+ */
public void cloneSnapshot(final String snapshotName, final String tableName)
throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
+ cloneSnapshot(snapshotName, TableName.valueOf(tableName));
+ }
+
+ /**
+ * Create a new table by cloning the snapshot content.
+ *
+ * @param snapshotName name of the snapshot to be cloned
+ * @param tableName name of the table where the snapshot will be restored
+ * @throws IOException if a remote or network exception occurs
+ * @throws TableExistsException if table to be created already exists
+ * @throws RestoreSnapshotException if snapshot failed to be cloned
+ * @throws IllegalArgumentException if the specified table has not a valid name
+ */
+ public void cloneSnapshot(final String snapshotName, final TableName tableName)
+ throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
if (tableExists(tableName)) {
- throw new TableExistsException("Table '" + tableName + " already exists");
+ throw new TableExistsException("Table " + tableName + " already exists");
}
internalRestoreSnapshot(snapshotName, tableName);
- waitUntilTableIsEnabled(Bytes.toBytes(tableName));
+ waitUntilTableIsEnabled(tableName);
}
/**
@@ -2485,10 +2776,11 @@
* @throws RestoreSnapshotException if snapshot failed to be restored
* @throws IllegalArgumentException if the restore request is formatted incorrectly
*/
- private void internalRestoreSnapshot(final String snapshotName, final String tableName)
+ private void internalRestoreSnapshot(final String snapshotName, final TableName
+ tableName)
throws IOException, RestoreSnapshotException {
SnapshotDescription snapshot = SnapshotDescription.newBuilder()
- .setName(snapshotName).setTable(tableName).build();
+ .setName(snapshotName).setTable(tableName.getNameAsString()).build();
// actually restore the snapshot
internalRestoreSnapshotAsync(snapshot);
@@ -2574,7 +2866,7 @@
public List listSnapshots(String regex) throws IOException {
return listSnapshots(Pattern.compile(regex));
}
-
+
/**
* List all the completed snapshots matching the given pattern.
*
@@ -2592,7 +2884,7 @@
}
return matched;
}
-
+
/**
* Delete an existing snapshot.
* @param snapshotName name of the snapshot
@@ -2609,7 +2901,7 @@
*/
public void deleteSnapshot(final String snapshotName) throws IOException {
// make sure the snapshot is possibly valid
- HTableDescriptor.isLegalTableName(Bytes.toBytes(snapshotName));
+ TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(snapshotName));
// do the delete
executeCallable(new MasterAdminCallable(getConnection()) {
@Override
@@ -2630,7 +2922,7 @@
public void deleteSnapshots(final String regex) throws IOException {
deleteSnapshots(Pattern.compile(regex));
}
-
+
/**
* Delete existing snapshots whose names match the pattern passed.
* @param pattern pattern for names of the snapshot to match
@@ -2652,7 +2944,7 @@
}
/**
- * @see {@link #execute(MasterAdminCallable)}
+ * @see {@link #executeCallable(org.apache.hadoop.hbase.client.HBaseAdmin.MasterCallable)}
*/
abstract static class MasterAdminCallable extends MasterCallable {
protected MasterAdminKeepAliveConnection masterAdmin;
@@ -2673,7 +2965,7 @@
}
/**
- * @see {@link #execute(MasterMonitorCallable)}
+ * @see {@link #executeCallable(org.apache.hadoop.hbase.client.HBaseAdmin.MasterCallable)}
*/
abstract static class MasterMonitorCallable extends MasterCallable {
protected MasterMonitorKeepAliveConnection masterMonitor;
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java (working copy)
@@ -27,6 +27,7 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
@@ -136,6 +137,9 @@
* @return true if the table is enabled, false otherwise
* @throws IOException if a remote or network exception occurs
*/
+ boolean isTableEnabled(TableName tableName) throws IOException;
+
+ @Deprecated
boolean isTableEnabled(byte[] tableName) throws IOException;
/**
@@ -143,6 +147,9 @@
* @return true if the table is disabled, false otherwise
* @throws IOException if a remote or network exception occurs
*/
+ boolean isTableDisabled(TableName tableName) throws IOException;
+
+ @Deprecated
boolean isTableDisabled(byte[] tableName) throws IOException;
/**
@@ -150,6 +157,9 @@
* @return true if all regions of the table are available, false otherwise
* @throws IOException if a remote or network exception occurs
*/
+ boolean isTableAvailable(TableName tableName) throws IOException;
+
+ @Deprecated
boolean isTableAvailable(byte[] tableName) throws IOException;
/**
@@ -164,8 +174,13 @@
* @throws IOException
* if a remote or network exception occurs
*/
- boolean isTableAvailable(byte[] tableName, byte[][] splitKeys) throws IOException;
+ boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws
+ IOException;
+ @Deprecated
+ boolean isTableAvailable(byte[] tableName, byte[][] splitKeys) throws
+ IOException;
+
/**
* List all the userspace tables. In other words, scan the META table.
*
@@ -183,6 +198,10 @@
* @return table metadata
* @throws IOException if a remote or network exception occurs
*/
+ HTableDescriptor getHTableDescriptor(TableName tableName)
+ throws IOException;
+
+ @Deprecated
HTableDescriptor getHTableDescriptor(byte[] tableName)
throws IOException;
@@ -195,11 +214,13 @@
* question
* @throws IOException if a remote or network exception occurs
*/
- HRegionLocation locateRegion(
- final byte[] tableName, final byte[] row
- )
- throws IOException;
+ public HRegionLocation locateRegion(final TableName tableName,
+ final byte [] row) throws IOException;
+ @Deprecated
+ public HRegionLocation locateRegion(final byte[] tableName,
+ final byte [] row) throws IOException;
+
/**
* Allows flushing the region cache.
*/
@@ -211,6 +232,9 @@
* @param tableName Name of the table whose regions we are to remove from
* cache.
*/
+ void clearRegionCache(final TableName tableName);
+
+ @Deprecated
void clearRegionCache(final byte[] tableName);
/**
@@ -228,11 +252,13 @@
* question
* @throws IOException if a remote or network exception occurs
*/
- HRegionLocation relocateRegion(
- final byte[] tableName, final byte[] row
- )
- throws IOException;
+ HRegionLocation relocateRegion(final TableName tableName,
+ final byte [] row) throws IOException;
+ @Deprecated
+ HRegionLocation relocateRegion(final byte[] tableName,
+ final byte [] row) throws IOException;
+
/**
* Update the location cache. This is used internally by HBase, in most cases it should not be
* used by the client application.
@@ -241,10 +267,13 @@
* @param exception the exception if any. Can be null.
* @param source the previous location
*/
- void updateCachedLocations(
- byte[] tableName, byte[] rowkey, Object exception, HRegionLocation source
- );
+ void updateCachedLocations(TableName tableName, byte[] rowkey,
+ Object exception, HRegionLocation source);
+ @Deprecated
+ void updateCachedLocations(byte[] tableName, byte[] rowkey,
+ Object exception, HRegionLocation source);
+
/**
* Gets the location of the region of regionName.
* @param regionName name of the region to locate
@@ -261,9 +290,11 @@
* @return list of region locations for all regions of table
* @throws IOException
*/
- List locateRegions(final byte[] tableName)
- throws IOException;
+ List locateRegions(final TableName tableName) throws IOException;
+ @Deprecated
+ List locateRegions(final byte[] tableName) throws IOException;
+
/**
* Gets the locations of all regions in the specified table, tableName.
* @param tableName table to get regions of
@@ -273,10 +304,15 @@
* @return list of region locations for all regions of table
* @throws IOException
*/
- List locateRegions(
- final byte[] tableName, final boolean useCache, final boolean offlined
- ) throws IOException;
+ public List locateRegions(final TableName tableName,
+ final boolean useCache,
+ final boolean offlined) throws IOException;
+ @Deprecated
+ public List locateRegions(final byte[] tableName,
+ final boolean useCache,
+ final boolean offlined) throws IOException;
+
/**
* Returns a {@link MasterAdminKeepAliveConnection} to the active master
*/
@@ -325,10 +361,15 @@
* @return Location of row.
* @throws IOException if a remote or network exception occurs
*/
- HRegionLocation getRegionLocation(byte [] tableName, byte [] row,
+ HRegionLocation getRegionLocation(TableName tableName, byte [] row,
boolean reload)
throws IOException;
+ @Deprecated
+ HRegionLocation getRegionLocation(byte[] tableName, byte [] row,
+ boolean reload)
+ throws IOException;
+
/**
* Process a mixed batch of Get, Put and Delete actions. All actions for a
* RegionServer are forwarded in one RPC call.
@@ -345,25 +386,32 @@
* @deprecated since 0.96 - Use {@link HTableInterface#batch} instead
*/
@Deprecated
- void processBatch(
- List extends Row> actions, final byte[] tableName, ExecutorService pool, Object[] results
- )
- throws IOException, InterruptedException;
+ void processBatch(List extends Row> actions, final TableName tableName,
+ ExecutorService pool, Object[] results) throws IOException, InterruptedException;
+ @Deprecated
+ void processBatch(List extends Row> actions, final byte[] tableName,
+ ExecutorService pool, Object[] results) throws IOException, InterruptedException;
+
/**
* Parameterized batch processing, allowing varying return types for different
* {@link Row} implementations.
* @deprecated since 0.96 - Use {@link HTableInterface#batchCallback} instead
*/
@Deprecated
- void processBatchCallback(
- List extends Row> list,
- byte[] tableName,
- ExecutorService pool,
- Object[] results,
- Batch.Callback callback
- ) throws IOException, InterruptedException;
+ public void processBatchCallback(List extends Row> list,
+ final TableName tableName,
+ ExecutorService pool,
+ Object[] results,
+ Batch.Callback callback) throws IOException, InterruptedException;
+ @Deprecated
+ public void processBatchCallback(List extends Row> list,
+ final byte[] tableName,
+ ExecutorService pool,
+ Object[] results,
+ Batch.Callback callback) throws IOException, InterruptedException;
+
/**
* Enable or disable region cache prefetch for the table. It will be
* applied for the given table's all HTable instances within this
@@ -371,16 +419,20 @@
* @param tableName name of table to configure.
* @param enable Set to true to enable region cache prefetch.
*/
- void setRegionCachePrefetch(
- final byte[] tableName, final boolean enable
- );
+ public void setRegionCachePrefetch(final TableName tableName,
+ final boolean enable);
+ public void setRegionCachePrefetch(final byte[] tableName,
+ final boolean enable);
+
/**
* Check whether region cache prefetch is enabled or not.
* @param tableName name of table to check
* @return true if table's region cache prefetch is enabled. Otherwise
* it is disabled.
*/
+ boolean getRegionCachePrefetch(final TableName tableName);
+
boolean getRegionCachePrefetch(final byte[] tableName);
/**
@@ -395,9 +447,12 @@
* @return HTD[] table metadata
* @throws IOException if a remote or network exception occurs
*/
- HTableDescriptor[] getHTableDescriptors(List tableNames)
- throws IOException;
+ HTableDescriptor[] getHTableDescriptorsByTableName(List tableNames) throws IOException;
+ @Deprecated
+ HTableDescriptor[] getHTableDescriptors(List tableNames) throws
+ IOException;
+
/**
* @return true if this connection is closed
*/
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (working copy)
@@ -48,6 +48,7 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Chore;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
@@ -72,6 +73,22 @@
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyNamespaceResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyNamespaceRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateNamespaceResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateNamespaceRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteNamespaceResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteNamespaceRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetNamespaceDescriptorResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetNamespaceDescriptorRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos
+ .ListNamespaceDescriptorsResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos
+ .ListNamespaceDescriptorsRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos
+ .GetTableDescriptorsByNamespaceResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos
+ .GetTableDescriptorsByNamespaceRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AddColumnRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AddColumnResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest;
@@ -442,7 +459,7 @@
* @return Number of cached regions for the table.
* @throws ZooKeeperConnectionException
*/
- static int getCachedRegionCount(Configuration conf, final byte[] tableName)
+ static int getCachedRegionCount(Configuration conf, final TableName tableName)
throws IOException {
return execute(new HConnectable(conf) {
@Override
@@ -458,7 +475,9 @@
* @return true if the region where the table and row reside is cached.
* @throws ZooKeeperConnectionException
*/
- static boolean isRegionCached(Configuration conf, final byte[] tableName, final byte[] row)
+ static boolean isRegionCached(Configuration conf,
+ final TableName tableName,
+ final byte[] row)
throws IOException {
return execute(new HConnectable(conf) {
@Override
@@ -542,11 +561,11 @@
private RpcClient rpcClient;
/**
- * Map of table to table {@link HRegionLocation}s. The table key is made
- * by doing a {@link Bytes#mapKey(byte[])} of the table's name.
- */
- private final Map> cachedRegionLocations =
- new HashMap>();
+ * Map of table to table {@link HRegionLocation}s.
+ */
+ private final Map>
+ cachedRegionLocations =
+ new HashMap>();
// The presence of a server in the map implies it's likely that there is an
// entry in cachedRegionLocations that map to this server; but the absence
@@ -792,24 +811,41 @@
}
@Override
- public HRegionLocation getRegionLocation(final byte [] name,
+ public HRegionLocation getRegionLocation(final TableName tableName,
final byte [] row, boolean reload)
throws IOException {
- return reload? relocateRegion(name, row): locateRegion(name, row);
+ return reload? relocateRegion(tableName, row): locateRegion(tableName, row);
}
@Override
- public boolean isTableEnabled(byte[] tableName) throws IOException {
+ public HRegionLocation getRegionLocation(final byte[] tableName,
+ final byte [] row, boolean reload)
+ throws IOException {
+ return getRegionLocation(TableName.valueOf(tableName), row, reload);
+ }
+
+ @Override
+ public boolean isTableEnabled(TableName tableName) throws IOException {
return this.registry.isTableOnlineState(tableName, true);
}
@Override
- public boolean isTableDisabled(byte[] tableName) throws IOException {
+ public boolean isTableEnabled(byte[] tableName) throws IOException {
+ return isTableEnabled(TableName.valueOf(tableName));
+ }
+
+ @Override
+ public boolean isTableDisabled(TableName tableName) throws IOException {
return this.registry.isTableOnlineState(tableName, false);
}
@Override
- public boolean isTableAvailable(final byte[] tableName) throws IOException {
+ public boolean isTableDisabled(byte[] tableName) throws IOException {
+ return isTableDisabled(TableName.valueOf(tableName));
+ }
+
+ @Override
+ public boolean isTableAvailable(final TableName tableName) throws IOException {
final AtomicBoolean available = new AtomicBoolean(true);
final AtomicInteger regionCount = new AtomicInteger(0);
MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
@@ -817,14 +853,15 @@
public boolean processRow(Result row) throws IOException {
HRegionInfo info = MetaScanner.getHRegionInfo(row);
if (info != null) {
- if (Bytes.compareTo(tableName, info.getTableName()) == 0) {
+ if (tableName.equals(info.getTableName())) {
ServerName server = HRegionInfo.getServerName(row);
if (server == null) {
available.set(false);
return false;
}
regionCount.incrementAndGet();
- } else if (Bytes.compareTo(tableName, info.getTableName()) < 0) {
+ } else if (tableName.compareTo(
+ info.getTableName()) < 0) {
// Return if we are done with the current table
return false;
}
@@ -837,7 +874,12 @@
}
@Override
- public boolean isTableAvailable(final byte[] tableName, final byte[][] splitKeys)
+ public boolean isTableAvailable(final byte[] tableName) throws IOException {
+ return isTableAvailable(TableName.valueOf(tableName));
+ }
+
+ @Override
+ public boolean isTableAvailable(final TableName tableName, final byte[][] splitKeys)
throws IOException {
final AtomicBoolean available = new AtomicBoolean(true);
final AtomicInteger regionCount = new AtomicInteger(0);
@@ -846,7 +888,7 @@
public boolean processRow(Result row) throws IOException {
HRegionInfo info = MetaScanner.getHRegionInfo(row);
if (info != null) {
- if (Bytes.compareTo(tableName, info.getTableName()) == 0) {
+ if (tableName.equals(info.getTableName())) {
ServerName server = HRegionInfo.getServerName(row);
if (server == null) {
available.set(false);
@@ -864,7 +906,7 @@
// Always empty start row should be counted
regionCount.incrementAndGet();
}
- } else if (Bytes.compareTo(tableName, info.getTableName()) < 0) {
+ } else if (tableName.compareTo(info.getTableName()) < 0) {
// Return if we are done with the current table
return false;
}
@@ -878,6 +920,12 @@
}
@Override
+ public boolean isTableAvailable(final byte[] tableName, final byte[][] splitKeys)
+ throws IOException {
+ return isTableAvailable(TableName.valueOf(tableName), splitKeys);
+ }
+
+ @Override
public HRegionLocation locateRegion(final byte[] regionName) throws IOException {
return locateRegion(HRegionInfo.getTableName(regionName),
HRegionInfo.getStartKey(regionName), false, true);
@@ -893,14 +941,20 @@
}
@Override
- public List locateRegions(final byte[] tableName)
+ public List locateRegions(final TableName tableName)
throws IOException {
return locateRegions (tableName, false, true);
}
@Override
- public List locateRegions(final byte[] tableName, final boolean useCache,
- final boolean offlined) throws IOException {
+ public List locateRegions(final byte[] tableName)
+ throws IOException {
+ return locateRegions(TableName.valueOf(tableName));
+ }
+
+ @Override
+ public List locateRegions(final TableName tableName,
+ final boolean useCache, final boolean offlined) throws IOException {
NavigableMap regions = MetaScanner.allTableRegions(conf, this,
tableName, offlined);
final List locations = new ArrayList();
@@ -911,41 +965,59 @@
}
@Override
- public HRegionLocation locateRegion(final byte [] tableName,
+ public List locateRegions(final byte[] tableName,
+ final boolean useCache, final boolean offlined) throws IOException {
+ return locateRegions(TableName.valueOf(tableName), useCache, offlined);
+ }
+
+ @Override
+ public HRegionLocation locateRegion(final TableName tableName,
final byte [] row)
throws IOException{
return locateRegion(tableName, row, true, true);
}
@Override
- public HRegionLocation relocateRegion(final byte [] tableName,
+ public HRegionLocation locateRegion(final byte[] tableName,
final byte [] row)
throws IOException{
+ return locateRegion(TableName.valueOf(tableName), row);
+ }
+ @Override
+ public HRegionLocation relocateRegion(final TableName tableName,
+ final byte [] row) throws IOException{
// Since this is an explicit request not to use any caching, finding
// disabled tables should not be desirable. This will ensure that an exception is thrown when
// the first time a disabled table is interacted with.
if (isTableDisabled(tableName)) {
- throw new DoNotRetryIOException(Bytes.toString(tableName) + " is disabled.");
+ throw new DoNotRetryIOException(tableName.getNameAsString() + " is disabled.");
}
return locateRegion(tableName, row, false, true);
}
- private HRegionLocation locateRegion(final byte [] tableName,
+ @Override
+ public HRegionLocation relocateRegion(final byte[] tableName,
+ final byte [] row) throws IOException {
+ return relocateRegion(TableName.valueOf(tableName), row);
+ }
+
+
+ private HRegionLocation locateRegion(final TableName tableName,
final byte [] row, boolean useCache, boolean retry)
throws IOException {
if (this.closed) throw new IOException(toString() + " closed");
- if (tableName == null || tableName.length == 0) {
+ if (tableName== null || tableName.getName().length == 0) {
throw new IllegalArgumentException(
"table name cannot be null or zero length");
}
- if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
+ if (tableName.equals(TableName.META_TABLE_NAME)) {
return this.registry.getMetaRegionLocation();
} else {
// Region not in the cache - have to go to the meta RS
- return locateRegionInMeta(HConstants.META_TABLE_NAME, tableName, row,
+ return locateRegionInMeta(TableName.META_TABLE_NAME, tableName, row,
useCache, userRegionLock, retry);
}
}
@@ -955,7 +1027,7 @@
* row we're seeking. It will prefetch certain number of regions info and
* save them to the global region cache.
*/
- private void prefetchRegionCache(final byte[] tableName,
+ private void prefetchRegionCache(final TableName tableName,
final byte[] row) {
// Implement a new visitor for MetaScanner, and use it to walk through
// the .META.
@@ -968,7 +1040,7 @@
}
// possible we got a region of a different table...
- if (!Bytes.equals(regionInfo.getTableName(), tableName)) {
+ if (!regionInfo.getTableName().equals(tableName)) {
return false; // stop scanning
}
if (regionInfo.isOffline()) {
@@ -994,7 +1066,7 @@
try {
// pre-fetch certain number of regions info at region cache.
MetaScanner.metaScan(conf, this, visitor, tableName, row,
- this.prefetchRegionLimit, HConstants.META_TABLE_NAME);
+ this.prefetchRegionLimit, TableName.META_TABLE_NAME);
} catch (IOException e) {
LOG.warn("Encountered problems when prefetch META table: ", e);
}
@@ -1004,8 +1076,8 @@
* Search the .META. table for the HRegionLocation
* info that contains the table and row we're seeking.
*/
- private HRegionLocation locateRegionInMeta(final byte [] parentTable,
- final byte [] tableName, final byte [] row, boolean useCache,
+ private HRegionLocation locateRegionInMeta(final TableName parentTable,
+ final TableName tableName, final byte [] row, boolean useCache,
Object regionLockObject, boolean retry)
throws IOException {
HRegionLocation location;
@@ -1051,7 +1123,7 @@
}
// If the parent table is META, we may want to pre-fetch some
// region info into the global region cache for this table.
- if (Bytes.equals(parentTable, HConstants.META_TABLE_NAME)
+ if (parentTable.equals(TableName.META_TABLE_NAME)
&& (getRegionCachePrefetch(tableName))) {
prefetchRegionCache(tableName, row);
}
@@ -1070,21 +1142,21 @@
HConstants.CATALOG_FAMILY);
}
if (regionInfoRow == null) {
- throw new TableNotFoundException(Bytes.toString(tableName));
+ throw new TableNotFoundException(tableName);
}
// convert the row result into the HRegionLocation we need!
HRegionInfo regionInfo = MetaScanner.getHRegionInfo(regionInfoRow);
if (regionInfo == null) {
throw new IOException("HRegionInfo was null or empty in " +
- Bytes.toString(parentTable) + ", row=" + regionInfoRow);
+ parentTable + ", row=" + regionInfoRow);
}
// possible we got a region of a different table...
- if (!Bytes.equals(regionInfo.getTableName(), tableName)) {
+ if (!regionInfo.getTableName().equals(tableName)) {
throw new TableNotFoundException(
- "Table '" + Bytes.toString(tableName) + "' was not found, got: " +
- Bytes.toString(regionInfo.getTableName()) + ".");
+ "Table '" + tableName + "' was not found, got: " +
+ regionInfo.getTableName() + ".");
}
if (regionInfo.isSplit()) {
throw new RegionOfflineException("the only available region for" +
@@ -1101,7 +1173,7 @@
ServerName serverName = HRegionInfo.getServerName(regionInfoRow);
if (serverName == null) {
throw new NoServerForRegionException("No server address listed " +
- "in " + Bytes.toString(parentTable) + " for region " +
+ "in " + parentTable + " for region " +
regionInfo.getRegionNameAsString() + " containing row " +
Bytes.toStringBinary(row));
}
@@ -1129,7 +1201,7 @@
if (tries < numTries - 1) {
if (LOG.isDebugEnabled()) {
LOG.debug("locateRegionInMeta parentTable=" +
- Bytes.toString(parentTable) + ", metaLocation=" +
+ parentTable + ", metaLocation=" +
((metaLocation == null)? "null": "{" + metaLocation + "}") +
", attempt=" + tries + " of " +
this.numTries + " failed; retrying after sleep of " +
@@ -1165,9 +1237,9 @@
* @param row
* @return Null or region location found in cache.
*/
- HRegionLocation getCachedLocation(final byte [] tableName,
+ HRegionLocation getCachedLocation(final TableName tableName,
final byte [] row) {
- SoftValueSortedMap tableLocations =
+ SoftValueSortedMap tableLocations =
getTableLocations(tableName);
// start to examine the cache. we can only do cache actions
@@ -1207,7 +1279,7 @@
* @param tableName tableName
* @param row
*/
- void forceDeleteCachedLocation(final byte [] tableName, final byte [] row) {
+ void forceDeleteCachedLocation(final TableName tableName, final byte [] row) {
HRegionLocation rl = null;
synchronized (this.cachedRegionLocations) {
Map tableLocations = getTableLocations(tableName);
@@ -1223,7 +1295,7 @@
if ((rl != null) && LOG.isDebugEnabled()) {
LOG.debug("Removed " + rl.getHostname() + ":" + rl.getPort()
+ " as a location of " + rl.getRegionInfo().getRegionNameAsString() +
- " for tableName=" + Bytes.toString(tableName) + " from cache");
+ " for tableName=" + tableName + " from cache");
}
}
@@ -1259,18 +1331,16 @@
* @param tableName
* @return Map of cached locations for passed tableName
*/
- private SoftValueSortedMap getTableLocations(
- final byte [] tableName) {
+ private SoftValueSortedMap getTableLocations(
+ final TableName tableName) {
// find the map of cached locations for this table
- Integer key = Bytes.mapKey(tableName);
- SoftValueSortedMap result;
+ SoftValueSortedMap result;
synchronized (this.cachedRegionLocations) {
- result = this.cachedRegionLocations.get(key);
+ result = this.cachedRegionLocations.get(tableName);
// if tableLocations for this table isn't built yet, make one
if (result == null) {
- result = new SoftValueSortedMap(
- Bytes.BYTES_COMPARATOR);
- this.cachedRegionLocations.put(key, result);
+ result = new SoftValueSortedMap(Bytes.BYTES_COMPARATOR);
+ this.cachedRegionLocations.put(tableName, result);
}
}
return result;
@@ -1285,23 +1355,28 @@
}
@Override
- public void clearRegionCache(final byte [] tableName) {
+ public void clearRegionCache(final TableName tableName) {
synchronized (this.cachedRegionLocations) {
- this.cachedRegionLocations.remove(Bytes.mapKey(tableName));
+ this.cachedRegionLocations.remove(tableName);
}
}
+ @Override
+ public void clearRegionCache(final byte[] tableName) {
+ clearRegionCache(TableName.valueOf(tableName));
+ }
+
/**
* Put a newly discovered HRegionLocation into the cache.
* @param tableName The table name.
* @param source the source of the new location, if it's not coming from meta
* @param location the new location
*/
- private void cacheLocation(final byte [] tableName, final HRegionLocation source,
+ private void cacheLocation(final TableName tableName, final HRegionLocation source,
final HRegionLocation location) {
boolean isFromMeta = (source == null);
byte [] startKey = location.getRegionInfo().getStartKey();
- Map tableLocations =
+ Map tableLocations =
getTableLocations(tableName);
boolean isNewCacheEntry = false;
boolean isStaleUpdate = false;
@@ -2004,6 +2079,36 @@
}
@Override
+ public ModifyNamespaceResponse modifyNamespace(RpcController controller, ModifyNamespaceRequest request) throws ServiceException {
+ return stub.modifyNamespace(controller, request);
+ }
+
+ @Override
+ public CreateNamespaceResponse createNamespace(RpcController controller, CreateNamespaceRequest request) throws ServiceException {
+ return stub.createNamespace(controller, request);
+ }
+
+ @Override
+ public DeleteNamespaceResponse deleteNamespace(RpcController controller, DeleteNamespaceRequest request) throws ServiceException {
+ return stub.deleteNamespace(controller, request);
+ }
+
+ @Override
+ public GetNamespaceDescriptorResponse getNamespaceDescriptor(RpcController controller, GetNamespaceDescriptorRequest request) throws ServiceException {
+ return stub.getNamespaceDescriptor(controller, request);
+ }
+
+ @Override
+ public ListNamespaceDescriptorsResponse listNamespaceDescriptors(RpcController controller, ListNamespaceDescriptorsRequest request) throws ServiceException {
+ return stub.listNamespaceDescriptors(controller, request);
+ }
+
+ @Override
+ public GetTableDescriptorsByNamespaceResponse getTableDescriptorsByNamespace(RpcController controller, GetTableDescriptorsByNamespaceRequest request) throws ServiceException {
+ return stub.getTableDescriptorsByNamespace(controller, request);
+ }
+
+ @Override
public void close() {
release(this.mss);
}
@@ -2147,8 +2252,9 @@
return;
}
synchronized (this.cachedRegionLocations) {
- byte[] tableName = location.getRegionInfo().getTableName();
- Map tableLocations = getTableLocations(tableName);
+ TableName tableName = location.getRegionInfo().getTableName();
+ Map tableLocations =
+ getTableLocations(tableName);
if (!tableLocations.isEmpty()) {
// Delete if there's something in the cache for this region.
HRegionLocation removedLocation =
@@ -2156,7 +2262,7 @@
if (LOG.isDebugEnabled() && removedLocation != null) {
LOG.debug("Removed " +
location.getRegionInfo().getRegionNameAsString() +
- " for tableName=" + Bytes.toString(tableName) +
+ " for tableName=" + tableName +
" from cache");
}
}
@@ -2171,11 +2277,11 @@
* @param source server that is the source of the location update.
*/
@Override
- public void updateCachedLocations(final byte[] tableName, byte[] rowkey,
+ public void updateCachedLocations(final TableName tableName, byte[] rowkey,
final Object exception, final HRegionLocation source) {
if (rowkey == null || tableName == null) {
LOG.warn("Coding error, see method javadoc. row=" + (rowkey == null ? "null" : rowkey) +
- ", tableName=" + (tableName == null ? "null" : Bytes.toString(tableName)));
+ ", tableName=" + (tableName == null ? "null" : tableName));
return;
}
@@ -2206,9 +2312,15 @@
}
@Override
+ public void updateCachedLocations(final byte[] tableName, byte[] rowkey,
+ final Object exception, final HRegionLocation source) {
+ updateCachedLocations(TableName.valueOf(tableName), rowkey, exception, source);
+ }
+
+ @Override
@Deprecated
public void processBatch(List extends Row> list,
- final byte[] tableName,
+ final TableName tableName,
ExecutorService pool,
Object[] results) throws IOException, InterruptedException {
// This belongs in HTable!!! Not in here. St.Ack
@@ -2221,6 +2333,15 @@
processBatchCallback(list, tableName, pool, results, null);
}
+ @Override
+ @Deprecated
+ public void processBatch(List extends Row> list,
+ final byte[] tableName,
+ ExecutorService pool,
+ Object[] results) throws IOException, InterruptedException {
+ processBatch(list, TableName.valueOf(tableName), pool, results);
+ }
+
/**
* Send the queries in parallel on the different region servers. Retries on failures.
* If the method returns it means that there is no error, and the 'results' array will
@@ -2232,7 +2353,7 @@
@Deprecated
public void processBatchCallback(
List extends Row> list,
- byte[] tableName,
+ TableName tableName,
ExecutorService pool,
Object[] results,
Batch.Callback callback)
@@ -2252,8 +2373,20 @@
}
}
+ @Override
+ @Deprecated
+ public void processBatchCallback(
+ List extends Row> list,
+ byte[] tableName,
+ ExecutorService pool,
+ Object[] results,
+ Batch.Callback callback)
+ throws IOException, InterruptedException {
+ processBatchCallback(list, TableName.valueOf(tableName), pool, results, callback);
+ }
+
// For tests.
- protected AsyncProcess createAsyncProcess(byte[] tableName, ExecutorService pool,
+ protected AsyncProcess createAsyncProcess(TableName tableName, ExecutorService pool,
AsyncProcess.AsyncProcessCallback callback, Configuration conf) {
return new AsyncProcess(this, tableName, pool, callback, conf,
RpcRetryingCallerFactory.instantiate(conf));
@@ -2302,10 +2435,9 @@
* Return the number of cached region for a table. It will only be called
* from a unit test.
*/
- int getNumberOfCachedRegionLocations(final byte[] tableName) {
- Integer key = Bytes.mapKey(tableName);
+ int getNumberOfCachedRegionLocations(final TableName tableName) {
synchronized (this.cachedRegionLocations) {
- Map tableLocs = this.cachedRegionLocations.get(key);
+ Map tableLocs = this.cachedRegionLocations.get(tableName);
if (tableLocs == null) {
return 0;
}
@@ -2320,28 +2452,39 @@
* @param row row
* @return Region cached or not.
*/
- boolean isRegionCached(final byte[] tableName, final byte[] row) {
+ boolean isRegionCached(TableName tableName, final byte[] row) {
HRegionLocation location = getCachedLocation(tableName, row);
return location != null;
}
@Override
- public void setRegionCachePrefetch(final byte[] tableName,
+ public void setRegionCachePrefetch(final TableName tableName,
final boolean enable) {
if (!enable) {
- regionCachePrefetchDisabledTables.add(Bytes.mapKey(tableName));
+ regionCachePrefetchDisabledTables.add(Bytes.mapKey(tableName.getName()));
}
else {
- regionCachePrefetchDisabledTables.remove(Bytes.mapKey(tableName));
+ regionCachePrefetchDisabledTables.remove(Bytes.mapKey(tableName.getName()));
}
}
@Override
- public boolean getRegionCachePrefetch(final byte[] tableName) {
- return !regionCachePrefetchDisabledTables.contains(Bytes.mapKey(tableName));
+ public void setRegionCachePrefetch(final byte[] tableName,
+ final boolean enable) {
+ setRegionCachePrefetch(TableName.valueOf(tableName), enable);
}
@Override
+ public boolean getRegionCachePrefetch(TableName tableName) {
+ return !regionCachePrefetchDisabledTables.contains(Bytes.mapKey(tableName.getName()));
+ }
+
+ @Override
+ public boolean getRegionCachePrefetch(byte[] tableName) {
+ return getRegionCachePrefetch(TableName.valueOf(tableName));
+ }
+
+ @Override
public void abort(final String msg, Throwable t) {
if (t instanceof KeeperException.SessionExpiredException
&& keepAliveZookeeper != null) {
@@ -2457,7 +2600,7 @@
MasterMonitorKeepAliveConnection master = getKeepAliveMasterMonitorService();
try {
GetTableDescriptorsRequest req =
- RequestConverter.buildGetTableDescriptorsRequest((List)null);
+ RequestConverter.buildGetTableDescriptorsRequest((List)null);
return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
@@ -2467,7 +2610,8 @@
}
@Override
- public HTableDescriptor[] getHTableDescriptors(List tableNames) throws IOException {
+ public HTableDescriptor[] getHTableDescriptorsByTableName(
+ List tableNames) throws IOException {
if (tableNames == null || tableNames.isEmpty()) return new HTableDescriptor[0];
MasterMonitorKeepAliveConnection master = getKeepAliveMasterMonitorService();
try {
@@ -2481,6 +2625,17 @@
}
}
+ @Override
+ public HTableDescriptor[] getHTableDescriptors(
+ List names) throws IOException {
+ List tableNames = new ArrayList(names.size());
+ for(String name : names) {
+ tableNames.add(TableName.valueOf(name));
+ }
+
+ return getHTableDescriptorsByTableName(tableNames);
+ }
+
/**
* Connects to the master to get the table descriptor.
* @param tableName table name
@@ -2489,10 +2644,10 @@
* is not found.
*/
@Override
- public HTableDescriptor getHTableDescriptor(final byte[] tableName)
+ public HTableDescriptor getHTableDescriptor(final TableName tableName)
throws IOException {
- if (tableName == null || tableName.length == 0) return null;
- if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
+ if (tableName == null) return null;
+ if (tableName.equals(TableName.META_TABLE_NAME)) {
return HTableDescriptor.META_TABLEDESC;
}
MasterMonitorKeepAliveConnection master = getKeepAliveMasterMonitorService();
@@ -2509,8 +2664,14 @@
if (!htds.getTableSchemaList().isEmpty()) {
return HTableDescriptor.convert(htds.getTableSchemaList().get(0));
}
- throw new TableNotFoundException(Bytes.toString(tableName));
+ throw new TableNotFoundException(tableName.getNameAsString());
}
+
+ @Override
+ public HTableDescriptor getHTableDescriptor(final byte[] tableName)
+ throws IOException {
+ return getHTableDescriptor(TableName.valueOf(tableName));
+ }
}
/**
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionWrapper.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionWrapper.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionWrapper.java (working copy)
@@ -19,6 +19,7 @@
import java.io.IOException;
import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.ExecutorService;
@@ -28,7 +29,9 @@
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
@@ -106,88 +109,146 @@
}
@Override
- public boolean isTableEnabled(byte[] tableName) throws IOException {
+ public boolean isTableEnabled(TableName tableName) throws IOException {
return hconnection.isTableEnabled(tableName);
}
@Override
- public boolean isTableDisabled(byte[] tableName) throws IOException {
+ public boolean isTableEnabled(byte[] tableName) throws IOException {
+ return isTableEnabled(TableName.valueOf(tableName));
+ }
+
+ @Override
+ public boolean isTableDisabled(TableName tableName) throws IOException {
return hconnection.isTableDisabled(tableName);
}
@Override
- public boolean isTableAvailable(byte[] tableName) throws IOException {
+ public boolean isTableDisabled(byte[] tableName) throws IOException {
+ return isTableDisabled(TableName.valueOf(tableName));
+ }
+
+ @Override
+ public boolean isTableAvailable(TableName tableName) throws IOException {
return hconnection.isTableAvailable(tableName);
}
@Override
- public boolean isTableAvailable(byte[] tableName, byte[][] splitKeys)
- throws IOException {
+ public boolean isTableAvailable(byte[] tableName) throws IOException {
+ return isTableAvailable(TableName.valueOf(tableName));
+ }
+
+ @Override
+ public boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException {
return hconnection.isTableAvailable(tableName, splitKeys);
}
@Override
+ public boolean isTableAvailable(byte[] tableName, byte[][] splitKeys) throws IOException {
+ return isTableAvailable(TableName.valueOf(tableName), splitKeys);
+ }
+
+ @Override
public HTableDescriptor[] listTables() throws IOException {
return hconnection.listTables();
}
@Override
- public HTableDescriptor getHTableDescriptor(byte[] tableName)
- throws IOException {
+ public HTableDescriptor getHTableDescriptor(TableName tableName) throws IOException {
return hconnection.getHTableDescriptor(tableName);
}
@Override
- public HRegionLocation locateRegion(byte[] tableName, byte[] row)
- throws IOException {
+ public HTableDescriptor getHTableDescriptor(byte[] tableName) throws IOException {
+ return getHTableDescriptor(TableName.valueOf(tableName));
+ }
+
+ @Override
+ public HRegionLocation locateRegion(TableName tableName, byte[] row) throws IOException {
return hconnection.locateRegion(tableName, row);
}
@Override
+ public HRegionLocation locateRegion(byte[] tableName, byte[] row) throws IOException {
+ return locateRegion(TableName.valueOf(tableName), row);
+ }
+
+ @Override
public void clearRegionCache() {
hconnection.clearRegionCache();
}
@Override
- public void clearRegionCache(byte[] tableName) {
+ public void clearRegionCache(TableName tableName) {
hconnection.clearRegionCache(tableName);
}
@Override
+ public void clearRegionCache(byte[] tableName) {
+ clearRegionCache(TableName.valueOf(tableName));
+ }
+
+ @Override
public void deleteCachedRegionLocation(HRegionLocation location) {
hconnection.deleteCachedRegionLocation(location);
}
@Override
- public HRegionLocation relocateRegion(byte[] tableName, byte[] row)
- throws IOException {
+ public HRegionLocation relocateRegion(TableName tableName, byte[] row) throws IOException {
return hconnection.relocateRegion(tableName, row);
}
@Override
- public void updateCachedLocations(byte[] tableName, byte[] rowkey,
- Object exception, HRegionLocation source) {
+ public HRegionLocation relocateRegion(byte[] tableName, byte[] row) throws IOException {
+ return relocateRegion(TableName.valueOf(tableName), row);
+ }
+
+ @Override
+ public void updateCachedLocations(TableName tableName,
+ byte[] rowkey,
+ Object exception,
+ HRegionLocation source) {
hconnection.updateCachedLocations(tableName, rowkey, exception, source);
}
@Override
+ public void updateCachedLocations(byte[] tableName,
+ byte[] rowkey,
+ Object exception,
+ HRegionLocation source) {
+ updateCachedLocations(TableName.valueOf(tableName), rowkey, exception, source);
+ }
+
+ @Override
public HRegionLocation locateRegion(byte[] regionName) throws IOException {
return hconnection.locateRegion(regionName);
}
@Override
- public List locateRegions(byte[] tableName)
- throws IOException {
+ public List locateRegions(TableName tableName) throws IOException {
return hconnection.locateRegions(tableName);
}
@Override
- public List locateRegions(byte[] tableName,
- boolean useCache, boolean offlined) throws IOException {
+ public List locateRegions(byte[] tableName) throws IOException {
+ return locateRegions(TableName.valueOf(tableName));
+ }
+
+ @Override
+ public List locateRegions(TableName tableName,
+ boolean useCache,
+ boolean offlined) throws IOException {
return hconnection.locateRegions(tableName, useCache, offlined);
}
@Override
+ public List locateRegions(byte[] tableName,
+ boolean useCache,
+ boolean offlined) throws IOException {
+ return locateRegions(TableName.valueOf(tableName));
+ }
+
+ @Override
public MasterAdminService.BlockingInterface getMasterAdmin() throws IOException {
return hconnection.getMasterAdmin();
}
@@ -237,47 +298,89 @@
}
@Override
- public HRegionLocation getRegionLocation(byte[] tableName, byte[] row,
- boolean reload) throws IOException {
+ public HRegionLocation getRegionLocation(TableName tableName,
+ byte[] row, boolean reload) throws IOException {
return hconnection.getRegionLocation(tableName, row, reload);
}
@Override
- public void processBatch(List extends Row> actions, byte[] tableName,
- ExecutorService pool, Object[] results) throws IOException,
- InterruptedException {
+ public HRegionLocation getRegionLocation(byte[] tableName,
+ byte[] row, boolean reload) throws IOException {
+ return getRegionLocation(TableName.valueOf(tableName), row, reload);
+ }
+
+ @Override
+ public void processBatch(List extends Row> actions, TableName tableName, ExecutorService pool,
+ Object[] results) throws IOException, InterruptedException {
hconnection.processBatch(actions, tableName, pool, results);
}
@Override
- public void processBatchCallback(List extends Row> list,
- byte[] tableName, ExecutorService pool, Object[] results,
- Callback callback) throws IOException, InterruptedException {
+ public void processBatch(List extends Row> actions, byte[] tableName, ExecutorService pool,
+ Object[] results) throws IOException, InterruptedException {
+ processBatch(actions, TableName.valueOf(tableName), pool, results);
+ }
+
+ @Override
+ public void processBatchCallback(List extends Row> list, TableName tableName,
+ ExecutorService pool,
+ Object[] results,
+ Callback callback)
+ throws IOException, InterruptedException {
hconnection.processBatchCallback(list, tableName, pool, results, callback);
}
@Override
- public void setRegionCachePrefetch(byte[] tableName, boolean enable) {
+ public void processBatchCallback(List extends Row> list, byte[] tableName,
+ ExecutorService pool,
+ Object[] results,
+ Callback callback)
+ throws IOException, InterruptedException {
+ processBatchCallback(list, TableName.valueOf(tableName), pool, results, callback);
+ }
+
+ @Override
+ public void setRegionCachePrefetch(TableName tableName, boolean enable) {
hconnection.setRegionCachePrefetch(tableName, enable);
}
@Override
- public boolean getRegionCachePrefetch(byte[] tableName) {
+ public void setRegionCachePrefetch(byte[] tableName, boolean enable) {
+ setRegionCachePrefetch(TableName.valueOf(tableName), enable);
+ }
+
+ @Override
+ public boolean getRegionCachePrefetch(TableName tableName) {
return hconnection.getRegionCachePrefetch(tableName);
}
@Override
+ public boolean getRegionCachePrefetch(byte[] tableName) {
+ return getRegionCachePrefetch(TableName.valueOf(tableName));
+ }
+
+ @Override
public int getCurrentNrHRS() throws IOException {
return hconnection.getCurrentNrHRS();
}
@Override
- public HTableDescriptor[] getHTableDescriptors(List tableNames)
- throws IOException {
- return hconnection.getHTableDescriptors(tableNames);
+ public HTableDescriptor[] getHTableDescriptorsByTableName(
+ List tableNames) throws IOException {
+ return hconnection.getHTableDescriptorsByTableName(tableNames);
}
@Override
+ public HTableDescriptor[] getHTableDescriptors(
+ List names) throws IOException {
+ List tableNames = new ArrayList(names.size());
+ for(String name : names) {
+ tableNames.add(TableName.valueOf(name));
+ }
+ return getHTableDescriptorsByTableName(tableNames);
+ }
+
+ @Override
public boolean isClosed() {
return hconnection.isClosed();
}
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java (working copy)
@@ -26,6 +26,7 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
@@ -51,7 +52,6 @@
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.hbase.util.Threads;
import java.io.Closeable;
@@ -119,7 +119,7 @@
public class HTable implements HTableInterface {
private static final Log LOG = LogFactory.getLog(HTable.class);
protected HConnection connection;
- private final byte [] tableName;
+ private final TableName tableName;
private volatile Configuration configuration;
protected List writeAsyncBuffer = new LinkedList();
private long writeBufferSize;
@@ -150,10 +150,9 @@
*/
public HTable(Configuration conf, final String tableName)
throws IOException {
- this(conf, Bytes.toBytes(tableName));
+ this(conf, TableName.valueOf(tableName));
}
-
/**
* Creates an object to access a HBase table.
* Shares zookeeper connection and other resources with other HTable instances
@@ -164,8 +163,25 @@
* @param tableName Name of the table.
* @throws IOException if a remote or network exception occurs
*/
- public HTable(Configuration conf, final byte [] tableName)
+ public HTable(Configuration conf, final byte[] tableName)
throws IOException {
+ this(conf, TableName.valueOf(tableName));
+ }
+
+
+
+ /**
+ * Creates an object to access a HBase table.
+ * Shares zookeeper connection and other resources with other HTable instances
+ * created with the same conf instance. Uses already-populated
+ * region cache if one is available, populated by any other HTable instances
+ * sharing this conf instance. Recommended.
+ * @param conf Configuration object to use.
+ * @param tableName table name pojo
+ * @throws IOException if a remote or network exception occurs
+ */
+ public HTable(Configuration conf, final TableName tableName)
+ throws IOException {
this.tableName = tableName;
this.cleanupPoolOnClose = this.cleanupConnectionOnClose = true;
if (conf == null) {
@@ -206,6 +222,23 @@
*/
public HTable(Configuration conf, final byte[] tableName, final ExecutorService pool)
throws IOException {
+ this(conf, TableName.valueOf(tableName), pool);
+ }
+
+ /**
+ * Creates an object to access a HBase table.
+ * Shares zookeeper connection and other resources with other HTable instances
+ * created with the same conf instance. Uses already-populated
+ * region cache if one is available, populated by any other HTable instances
+ * sharing this conf instance.
+ * Use this constructor when the ExecutorService is externally managed.
+ * @param conf Configuration object to use.
+ * @param tableName Name of the table.
+ * @param pool ExecutorService to be used.
+ * @throws IOException if a remote or network exception occurs
+ */
+ public HTable(Configuration conf, final TableName tableName, final ExecutorService pool)
+ throws IOException {
this.connection = HConnectionManager.getConnection(conf);
this.configuration = conf;
this.pool = pool;
@@ -229,6 +262,22 @@
*/
public HTable(final byte[] tableName, final HConnection connection,
final ExecutorService pool) throws IOException {
+ this(TableName.valueOf(tableName), connection, pool);
+ }
+
+ /**
+ * Creates an object to access a HBase table.
+ * Shares zookeeper connection and other resources with other HTable instances
+ * created with the same connection instance.
+ * Use this constructor when the ExecutorService and HConnection instance are
+ * externally managed.
+ * @param tableName Name of the table.
+ * @param connection HConnection to be used.
+ * @param pool ExecutorService to be used.
+ * @throws IOException if a remote or network exception occurs
+ */
+ public HTable(TableName tableName, final HConnection connection,
+ final ExecutorService pool) throws IOException {
if (connection == null || connection.isClosed()) {
throw new IllegalArgumentException("Connection is null or closed.");
}
@@ -245,7 +294,7 @@
* For internal testing.
*/
protected HTable(){
- tableName = new byte[]{};
+ tableName = null;
cleanupPoolOnClose = false;
cleanupConnectionOnClose = false;
}
@@ -255,7 +304,7 @@
*/
private void finishSetup() throws IOException {
this.connection.locateRegion(tableName, HConstants.EMPTY_START_ROW);
- this.operationTimeout = HTableDescriptor.isMetaTable(tableName) ?
+ this.operationTimeout = HTableDescriptor.isSystemTable(tableName) ?
this.configuration.getInt(HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT,
HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT):
this.configuration.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
@@ -299,7 +348,7 @@
*/
@Deprecated
public static boolean isTableEnabled(String tableName) throws IOException {
- return isTableEnabled(Bytes.toBytes(tableName));
+ return isTableEnabled(TableName.valueOf(tableName));
}
/**
@@ -309,10 +358,24 @@
* @param tableName Name of table to check.
* @return {@code true} if table is online.
* @throws IOException if a remote or network exception occurs
+ * @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])}
+ */
+ @Deprecated
+ public static boolean isTableEnabled(byte[] tableName) throws IOException {
+ return isTableEnabled(TableName.valueOf(tableName));
+ }
+
+ /**
+ * Tells whether or not a table is enabled or not. This method creates a
+ * new HBase configuration, so it might make your unit tests fail due to
+ * incorrect ZK client port.
+ * @param tableName Name of table to check.
+ * @return {@code true} if table is online.
+ * @throws IOException if a remote or network exception occurs
* @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])}
*/
@Deprecated
- public static boolean isTableEnabled(byte[] tableName) throws IOException {
+ public static boolean isTableEnabled(TableName tableName) throws IOException {
return isTableEnabled(HBaseConfiguration.create(), tableName);
}
@@ -327,7 +390,7 @@
@Deprecated
public static boolean isTableEnabled(Configuration conf, String tableName)
throws IOException {
- return isTableEnabled(conf, Bytes.toBytes(tableName));
+ return isTableEnabled(conf, TableName.valueOf(tableName));
}
/**
@@ -336,11 +399,25 @@
* @param tableName Name of table to check.
* @return {@code true} if table is online.
* @throws IOException if a remote or network exception occurs
- * @deprecated use {@link HBaseAdmin#isTableEnabled(byte[] tableName)}
+ * @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])}
*/
@Deprecated
+ public static boolean isTableEnabled(Configuration conf, byte[] tableName)
+ throws IOException {
+ return isTableEnabled(conf, TableName.valueOf(tableName));
+ }
+
+ /**
+ * Tells whether or not a table is enabled or not.
+ * @param conf The Configuration object to use.
+ * @param tableName Name of table to check.
+ * @return {@code true} if table is online.
+ * @throws IOException if a remote or network exception occurs
+ * @deprecated use {@link HBaseAdmin#isTableEnabled(org.apache.hadoop.hbase.TableName tableName)}
+ */
+ @Deprecated
public static boolean isTableEnabled(Configuration conf,
- final byte[] tableName) throws IOException {
+ final TableName tableName) throws IOException {
return HConnectionManager.execute(new HConnectable(conf) {
@Override
public Boolean connect(HConnection connection) throws IOException {
@@ -388,9 +465,14 @@
*/
@Override
public byte [] getTableName() {
- return this.tableName;
+ return this.tableName.getName();
}
+ @Override
+ public TableName getName() {
+ return tableName;
+ }
+
/**
* INTERNAL Used by unit tests and tools to do low-level
* manipulations.
@@ -502,7 +584,7 @@
*/
public NavigableMap getRegionLocations() throws IOException {
// TODO: Odd that this returns a Map of HRI to SN whereas getRegionLocation, singular, returns an HRegionLocation.
- return MetaScanner.allTableRegions(getConfiguration(), this.connection, getTableName(), false);
+ return MetaScanner.allTableRegions(getConfiguration(), this.connection, getName(), false);
}
/**
@@ -611,7 +693,8 @@
if (scan.getCaching() <= 0) {
scan.setCaching(getScannerCaching());
}
- return new ClientScanner(getConfiguration(), scan, getTableName(), this.connection);
+ return new ClientScanner(getConfiguration(), scan,
+ getName(), this.connection);
}
/**
@@ -641,7 +724,7 @@
@Override
public Result get(final Get get) throws IOException {
RegionServerCallable callable = new RegionServerCallable(this.connection,
- getTableName(), get.getRow()) {
+ getName(), get.getRow()) {
public Result call() throws IOException {
return ProtobufUtil.get(getStub(), getLocation().getRegionInfo().getRegionName(), get);
}
@@ -813,7 +896,7 @@
if (synchronous || ap.hasError()) {
if (ap.hasError() && LOG.isDebugEnabled()) {
- LOG.debug(Bytes.toString(tableName) + ": One or more of the operations have failed -" +
+ LOG.debug(tableName + ": One or more of the operations have failed -" +
" waiting for all operation in progress to finish (successfully or not)");
}
ap.waitUntilDone();
@@ -845,7 +928,7 @@
@Override
public void mutateRow(final RowMutations rm) throws IOException {
RegionServerCallable callable =
- new RegionServerCallable(connection, getTableName(), rm.getRow()) {
+ new RegionServerCallable(connection, getName(), rm.getRow()) {
public Void call() throws IOException {
try {
MultiRequest request = RequestConverter.buildMultiRequest(
@@ -870,7 +953,7 @@
"Invalid arguments to append, no columns specified");
}
RegionServerCallable callable =
- new RegionServerCallable(this.connection, getTableName(), append.getRow()) {
+ new RegionServerCallable(this.connection, getName(), append.getRow()) {
public Result call() throws IOException {
try {
MutateRequest request = RequestConverter.buildMutateRequest(
@@ -897,7 +980,7 @@
"Invalid arguments to increment, no columns specified");
}
RegionServerCallable callable = new RegionServerCallable(this.connection,
- getTableName(), increment.getRow()) {
+ getName(), increment.getRow()) {
public Result call() throws IOException {
try {
MutateRequest request = RequestConverter.buildMutateRequest(
@@ -944,7 +1027,7 @@
}
RegionServerCallable callable =
- new RegionServerCallable(connection, getTableName(), row) {
+ new RegionServerCallable(connection, getName(), row) {
public Long call() throws IOException {
try {
MutateRequest request = RequestConverter.buildMutateRequest(
@@ -972,7 +1055,7 @@
final Put put)
throws IOException {
RegionServerCallable callable =
- new RegionServerCallable(connection, getTableName(), row) {
+ new RegionServerCallable(connection, getName(), row) {
public Boolean call() throws IOException {
try {
MutateRequest request = RequestConverter.buildMutateRequest(
@@ -998,7 +1081,7 @@
final Delete delete)
throws IOException {
RegionServerCallable callable =
- new RegionServerCallable(connection, getTableName(), row) {
+ new RegionServerCallable(connection, getName(), row) {
public Boolean call() throws IOException {
try {
MutateRequest request = RequestConverter.buildMutateRequest(
@@ -1020,7 +1103,7 @@
@Override
public boolean exists(final Get get) throws IOException {
RegionServerCallable callable =
- new RegionServerCallable(connection, getTableName(), get.getRow()) {
+ new RegionServerCallable(connection, getName(), get.getRow()) {
public Boolean call() throws IOException {
try {
GetRequest request = RequestConverter.buildGetRequest(
@@ -1124,7 +1207,7 @@
Callable> callable = new Callable>() {
public List call() throws Exception {
RegionServerCallable> callable =
- new RegionServerCallable>(connection, getTableName(),
+ new RegionServerCallable>(connection, getName(),
getsByRegionEntry.getValue().get(0).getRow()) {
public List call() throws IOException {
try {
@@ -1139,7 +1222,7 @@
}
};
return rpcCallerFactory.> newCaller().callWithRetries(callable,
- operationTimeout);
+ operationTimeout);
}
};
futures.put(getsByRegionEntry.getKey(), pool.submit(callable));
@@ -1352,6 +1435,12 @@
*/
public static void setRegionCachePrefetch(final byte[] tableName,
final boolean enable) throws IOException {
+ setRegionCachePrefetch(TableName.valueOf(tableName), enable);
+ }
+
+ public static void setRegionCachePrefetch(
+ final TableName tableName,
+ final boolean enable) throws IOException {
HConnectionManager.execute(new HConnectable(HBaseConfiguration
.create()) {
@Override
@@ -1374,6 +1463,12 @@
*/
public static void setRegionCachePrefetch(final Configuration conf,
final byte[] tableName, final boolean enable) throws IOException {
+ setRegionCachePrefetch(conf, TableName.valueOf(tableName), enable);
+ }
+
+ public static void setRegionCachePrefetch(final Configuration conf,
+ final TableName tableName,
+ final boolean enable) throws IOException {
HConnectionManager.execute(new HConnectable(conf) {
@Override
public Void connect(HConnection connection) throws IOException {
@@ -1393,6 +1488,11 @@
*/
public static boolean getRegionCachePrefetch(final Configuration conf,
final byte[] tableName) throws IOException {
+ return getRegionCachePrefetch(conf, TableName.valueOf(tableName));
+ }
+
+ public static boolean getRegionCachePrefetch(final Configuration conf,
+ final TableName tableName) throws IOException {
return HConnectionManager.execute(new HConnectable(conf) {
@Override
public Boolean connect(HConnection connection) throws IOException {
@@ -1409,6 +1509,11 @@
* @throws IOException
*/
public static boolean getRegionCachePrefetch(final byte[] tableName) throws IOException {
+ return getRegionCachePrefetch(TableName.valueOf(tableName));
+ }
+
+ public static boolean getRegionCachePrefetch(
+ final TableName tableName) throws IOException {
return HConnectionManager.execute(new HConnectable(
HBaseConfiguration.create()) {
@Override
@@ -1416,7 +1521,7 @@
return connection.getRegionCachePrefetch(tableName);
}
});
- }
+ }
/**
* Explicitly clears the region cache to fetch the latest value from META.
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java (working copy)
@@ -23,6 +23,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
@@ -50,6 +51,11 @@
byte[] getTableName();
/**
+ * Gets the fully qualified table name instance of this table.
+ */
+ TableName getName();
+
+ /**
* Returns the {@link Configuration} object used by this instance.
*
* The reference returned is not a copy, so any change made to it will
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java (working copy)
@@ -25,6 +25,7 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
@@ -67,7 +68,7 @@
static final String TABLE_MULTIPLEXER_FLUSH_FREQ_MS = "hbase.tablemultiplexer.flush.frequency.ms";
- private Map tableNameToHTableMap;
+ private Map tableNameToHTableMap;
/** The map between each region server to its corresponding buffer queue */
private Map>
@@ -92,7 +93,7 @@
this.serverToBufferQueueMap = new ConcurrentHashMap>();
this.serverToFlushWorkerMap = new ConcurrentHashMap();
- this.tableNameToHTableMap = new ConcurrentSkipListMap(Bytes.BYTES_COMPARATOR);
+ this.tableNameToHTableMap = new ConcurrentSkipListMap();
this.retryNum = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
this.perRegionServerBufferQueueSize = perRegionServerBufferQueueSize;
@@ -101,24 +102,28 @@
/**
* The put request will be buffered by its corresponding buffer queue. Return false if the queue
* is already full.
- * @param table
+ * @param tableName
* @param put
* @return true if the request can be accepted by its corresponding buffer queue.
* @throws IOException
*/
- public boolean put(final byte[] table, final Put put) throws IOException {
- return put(table, put, this.retryNum);
+ public boolean put(TableName tableName, final Put put) throws IOException {
+ return put(tableName, put, this.retryNum);
}
+ public boolean put(byte[] tableName, final Put put) throws IOException {
+ return put(TableName.valueOf(tableName), put);
+ }
+
/**
* The puts request will be buffered by their corresponding buffer queue.
* Return the list of puts which could not be queued.
- * @param table
+ * @param tableName
* @param puts
* @return the list of puts which could not be queued
* @throws IOException
*/
- public List put(final byte[] table, final List puts)
+ public List put(TableName tableName, final List puts)
throws IOException {
if (puts == null)
return null;
@@ -126,7 +131,7 @@
List failedPuts = null;
boolean result;
for (Put put : puts) {
- result = put(table, put, this.retryNum);
+ result = put(tableName, put, this.retryNum);
if (result == false) {
// Create the failed puts list if necessary
@@ -140,24 +145,29 @@
return failedPuts;
}
+ public List put(byte[] tableName, final List puts) throws IOException {
+ return put(TableName.valueOf(tableName), puts);
+ }
+
+
/**
* The put request will be buffered by its corresponding buffer queue. And the put request will be
* retried before dropping the request.
* Return false if the queue is already full.
- * @param table
+ * @param tableName
* @param put
* @param retry
* @return true if the request can be accepted by its corresponding buffer queue.
* @throws IOException
*/
- public boolean put(final byte[] table, final Put put, int retry)
+ public boolean put(final TableName tableName, final Put put, int retry)
throws IOException {
if (retry <= 0) {
return false;
}
LinkedBlockingQueue queue;
- HTable htable = getHTable(table);
+ HTable htable = getHTable(tableName);
try {
htable.validatePut(put);
HRegionLocation loc = htable.getRegionLocation(put.getRow(), false);
@@ -175,6 +185,11 @@
return false;
}
+ public boolean put(final byte[] tableName, final Put put, int retry)
+ throws IOException {
+ return put(TableName.valueOf(tableName), put, retry);
+ }
+
/**
* @return the current HTableMultiplexerStatus
*/
@@ -183,14 +198,14 @@
}
- private HTable getHTable(final byte[] table) throws IOException {
- HTable htable = this.tableNameToHTableMap.get(table);
+ private HTable getHTable(TableName tableName) throws IOException {
+ HTable htable = this.tableNameToHTableMap.get(tableName);
if (htable == null) {
synchronized (this.tableNameToHTableMap) {
- htable = this.tableNameToHTableMap.get(table);
+ htable = this.tableNameToHTableMap.get(tableName);
if (htable == null) {
- htable = new HTable(conf, table);
- this.tableNameToHTableMap.put(table, htable);
+ htable = new HTable(conf, tableName);
+ this.tableNameToHTableMap.put(tableName, htable);
}
}
}
@@ -435,7 +450,7 @@
HRegionLocation oldLoc) throws IOException {
Put failedPut = failedPutStatus.getPut();
// The currentPut is failed. So get the table name for the currentPut.
- byte[] tableName = failedPutStatus.getRegionInfo().getTableName();
+ TableName tableName = failedPutStatus.getRegionInfo().getTableName();
// Decrease the retry count
int retryCount = failedPutStatus.getRetryCount() - 1;
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java (working copy)
@@ -23,6 +23,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
@@ -341,6 +342,11 @@
}
@Override
+ public TableName getName() {
+ return table.getName();
+ }
+
+ @Override
public Configuration getConfiguration() {
checkState();
return table.getConfiguration();
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java (working copy)
@@ -30,6 +30,7 @@
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
@@ -76,10 +77,10 @@
* @throws IOException e
*/
public static void metaScan(Configuration configuration, HConnection connection,
- MetaScannerVisitor visitor, byte [] userTableName)
+ MetaScannerVisitor visitor, TableName userTableName)
throws IOException {
metaScan(configuration, connection, visitor, userTableName, null, Integer.MAX_VALUE,
- HConstants.META_TABLE_NAME);
+ TableName.META_TABLE_NAME);
}
/**
@@ -98,11 +99,11 @@
* @throws IOException e
*/
public static void metaScan(Configuration configuration,
- MetaScannerVisitor visitor, byte [] userTableName, byte[] row,
+ MetaScannerVisitor visitor, TableName userTableName, byte[] row,
int rowLimit)
throws IOException {
metaScan(configuration, null, visitor, userTableName, row, rowLimit,
- HConstants.META_TABLE_NAME);
+ TableName.META_TABLE_NAME);
}
/**
@@ -123,15 +124,15 @@
* @throws IOException e
*/
public static void metaScan(Configuration configuration, HConnection connection,
- final MetaScannerVisitor visitor, final byte[] tableName,
- final byte[] row, final int rowLimit, final byte[] metaTableName)
+ final MetaScannerVisitor visitor, final TableName tableName,
+ final byte[] row, final int rowLimit, final TableName metaTableName)
throws IOException {
int rowUpperLimit = rowLimit > 0 ? rowLimit: Integer.MAX_VALUE;
HTable metaTable;
if (connection == null) {
- metaTable = new HTable(configuration, HConstants.META_TABLE_NAME, null);
+ metaTable = new HTable(configuration, TableName.META_TABLE_NAME, null);
} else {
- metaTable = new HTable(HConstants.META_TABLE_NAME, connection, null);
+ metaTable = new HTable(TableName.META_TABLE_NAME, connection, null);
}
// Calculate startrow for scan.
byte[] startRow;
@@ -142,17 +143,18 @@
byte[] searchRow = HRegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
Result startRowResult = metaTable.getRowOrBefore(searchRow, HConstants.CATALOG_FAMILY);
if (startRowResult == null) {
- throw new TableNotFoundException("Cannot find row in .META. for table: " +
- Bytes.toString(tableName) + ", row=" + Bytes.toStringBinary(searchRow));
+ throw new TableNotFoundException("Cannot find row in "+ TableName
+ .META_TABLE_NAME.getNameAsString()+" for table: "
+ + tableName + ", row=" + Bytes.toStringBinary(searchRow));
}
HRegionInfo regionInfo = getHRegionInfo(startRowResult);
if (regionInfo == null) {
throw new IOException("HRegionInfo was null or empty in Meta for " +
- Bytes.toString(tableName) + ", row=" + Bytes.toStringBinary(searchRow));
+ tableName + ", row=" + Bytes.toStringBinary(searchRow));
}
byte[] rowBefore = regionInfo.getStartKey();
startRow = HRegionInfo.createRegionName(tableName, rowBefore, HConstants.ZEROES, false);
- } else if (tableName == null || tableName.length == 0) {
+ } else if (tableName == null || tableName.getName().length == 0) {
// Full META scan
startRow = HConstants.EMPTY_START_ROW;
} else {
@@ -165,7 +167,7 @@
HConstants.DEFAULT_HBASE_META_SCANNER_CACHING));
scan.setCaching(rows);
if (LOG.isTraceEnabled()) {
- LOG.trace("Scanning " + Bytes.toString(metaTableName) + " starting at row=" +
+ LOG.trace("Scanning " + metaTableName.getNameAsString() + " starting at row=" +
Bytes.toStringBinary(startRow) + " for max=" + rowUpperLimit + " with caching=" + rows);
}
// Run the scan
@@ -267,11 +269,11 @@
* @throws IOException
*/
public static NavigableMap allTableRegions(Configuration conf,
- HConnection connection,
- final byte [] tablename, final boolean offlined) throws IOException {
+ HConnection connection, final TableName tableName,
+ final boolean offlined) throws IOException {
final NavigableMap regions =
new TreeMap();
- MetaScannerVisitor visitor = new TableMetaScannerVisitor(tablename) {
+ MetaScannerVisitor visitor = new TableMetaScannerVisitor(tableName) {
@Override
public boolean processRowInternal(Result rowResult) throws IOException {
HRegionInfo info = getHRegionInfo(rowResult);
@@ -280,7 +282,7 @@
return true;
}
};
- metaScan(conf, connection, visitor, tablename);
+ metaScan(conf, connection, visitor, tableName);
return regions;
}
@@ -340,9 +342,9 @@
* META entries for daughters are available during splits.
*/
public static abstract class TableMetaScannerVisitor extends DefaultMetaScannerVisitor {
- private byte[] tableName;
+ private TableName tableName;
- public TableMetaScannerVisitor(byte[] tableName) {
+ public TableMetaScannerVisitor(TableName tableName) {
super();
this.tableName = tableName;
}
@@ -353,7 +355,7 @@
if (info == null) {
return true;
}
- if (!(Bytes.equals(info.getTableName(), tableName))) {
+ if (!(info.getTableName().equals(tableName))) {
return false;
}
return super.processRow(rowResult);
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java (working copy)
@@ -23,6 +23,7 @@
import java.util.Map;
import org.apache.hadoop.hbase.CellScannable;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -42,7 +43,7 @@
class MultiServerCallable extends RegionServerCallable {
private final MultiAction multi;
- MultiServerCallable(final HConnection connection, final byte [] tableName,
+ MultiServerCallable(final HConnection connection, final TableName tableName,
final HRegionLocation location, final MultiAction multi) {
super(connection, tableName, null);
this.multi = multi;
@@ -120,4 +121,4 @@
// Use the location we were given in the constructor rather than go look it up.
setStub(getConnection().getClient(getLocation().getServerName()));
}
-}
\ No newline at end of file
+}
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java (working copy)
@@ -29,6 +29,7 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.NotServingRegionException;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.exceptions.RegionMovedException;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.util.Bytes;
@@ -44,7 +45,7 @@
// Public because used outside of this package over in ipc.
static final Log LOG = LogFactory.getLog(RegionServerCallable.class);
private final HConnection connection;
- private final byte [] tableName;
+ private final TableName tableName;
private final byte [] row;
private HRegionLocation location;
private ClientService.BlockingInterface stub;
@@ -56,7 +57,7 @@
* @param tableName Table name to which row belongs.
* @param row The row we want in tableName.
*/
- public RegionServerCallable(HConnection connection, byte [] tableName, byte [] row) {
+ public RegionServerCallable(HConnection connection, TableName tableName, byte [] row) {
this.connection = connection;
this.tableName = tableName;
this.row = row;
@@ -71,7 +72,7 @@
public void prepare(final boolean reload) throws IOException {
this.location = connection.getRegionLocation(tableName, row, reload);
if (this.location == null) {
- throw new IOException("Failed to find location, tableName=" + Bytes.toString(tableName) +
+ throw new IOException("Failed to find location, tableName=" + tableName +
", row=" + Bytes.toString(row) + ", reload=" + reload);
}
setStub(getConnection().getClient(getLocation().getServerName()));
@@ -100,7 +101,7 @@
this.location = location;
}
- public byte [] getTableName() {
+ public TableName getTableName() {
return this.tableName;
}
@@ -129,7 +130,7 @@
@Override
public String getExceptionMessageAdditionalDetail() {
- return "row '" + Bytes.toString(row) + "' on table '" + Bytes.toString(tableName);
+ return "row '" + Bytes.toString(row) + "' on table '" + tableName;
}
@Override
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/Registry.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/Registry.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/Registry.java (working copy)
@@ -19,6 +19,7 @@
import java.io.IOException;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionLocation;
/**
@@ -46,7 +47,7 @@
* @param enabled Return true if table is enabled
* @throws IOException
*/
- boolean isTableOnlineState(byte [] tableName, boolean enabled) throws IOException;
+ boolean isTableOnlineState(TableName tableName, boolean enabled) throws IOException;
/**
* @return Count of 'running' regionservers
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java (working copy)
@@ -27,6 +27,7 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
@@ -89,7 +90,7 @@
* @param scanMetrics the ScanMetrics to used, if it is null, ScannerCallable
* won't collect metrics
*/
- public ScannerCallable (HConnection connection, byte [] tableName, Scan scan,
+ public ScannerCallable (HConnection connection, TableName tableName, Scan scan,
ScanMetrics scanMetrics) {
super(connection, tableName, scan.getStartRow());
this.scan = scan;
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java (working copy)
@@ -40,7 +40,7 @@
* @param desc
*/
UnmodifyableHTableDescriptor(final HTableDescriptor desc) {
- super(desc.getName(), getUnmodifyableFamilies(desc), desc.getValues());
+ super(desc.getTableName(), getUnmodifyableFamilies(desc), desc.getValues());
}
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java (working copy)
@@ -21,10 +21,10 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKTableReadOnly;
@@ -95,15 +95,14 @@
}
@Override
- public boolean isTableOnlineState(byte [] tableName, boolean enabled)
+ public boolean isTableOnlineState(TableName tableName, boolean enabled)
throws IOException {
- String tableNameStr = Bytes.toString(tableName);
ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher();
try {
if (enabled) {
- return ZKTableReadOnly.isEnabledTable(zkw, tableNameStr);
+ return ZKTableReadOnly.isEnabledTable(zkw, tableName);
}
- return ZKTableReadOnly.isDisabledTable(zkw, tableNameStr);
+ return ZKTableReadOnly.isDisabledTable(zkw, tableName);
} catch (KeeperException e) {
throw new IOException("Enable/Disable failed", e);
} finally {
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/SecureBulkLoadClient.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/SecureBulkLoadClient.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/SecureBulkLoadClient.java (working copy)
@@ -20,10 +20,12 @@
import com.google.protobuf.ByteString;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos;
import org.apache.hadoop.hbase.security.SecureBulkLoadUtil;
@@ -45,7 +47,7 @@
this.table = table;
}
- public String prepareBulkLoad(final byte[] tableName) throws IOException {
+ public String prepareBulkLoad(final TableName tableName) throws IOException {
try {
return
table.coprocessorService(SecureBulkLoadProtos.SecureBulkLoadService.class,
@@ -61,7 +63,7 @@
SecureBulkLoadProtos.PrepareBulkLoadRequest request =
SecureBulkLoadProtos.PrepareBulkLoadRequest.newBuilder()
- .setTableName(com.google.protobuf.ByteString.copyFrom(tableName)).build();
+ .setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
instance.prepareBulkLoad(controller,
request,
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java (working copy)
@@ -23,6 +23,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.RegionServerCallable;
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
@@ -48,13 +49,13 @@
private static Log LOG = LogFactory.getLog(RegionCoprocessorRpcChannel.class);
private final HConnection connection;
- private final byte[] table;
+ private final TableName table;
private final byte[] row;
private byte[] lastRegion;
private RpcRetryingCallerFactory rpcFactory;
- public RegionCoprocessorRpcChannel(HConnection conn, byte[] table, byte[] row) {
+ public RegionCoprocessorRpcChannel(HConnection conn, TableName table, byte[] row) {
this.connection = conn;
this.table = table;
this.row = row;
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (working copy)
@@ -40,6 +40,7 @@
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
@@ -47,6 +48,7 @@
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
@@ -1563,7 +1565,7 @@
* @return the converted Permission
*/
public static Permission toPermission(AccessControlProtos.Permission proto) {
- if (proto.hasTable()) {
+ if (proto.hasTableName()) {
return toTablePermission(proto);
} else {
List actions = toPermissionActions(proto.getActionList());
@@ -1582,9 +1584,9 @@
byte[] qualifier = null;
byte[] family = null;
- byte[] table = null;
+ TableName table = null;
- if (proto.hasTable()) table = proto.getTable().toByteArray();
+ if (proto.hasTableName()) table = ProtobufUtil.toTableName(proto.getTableName());
if (proto.hasFamily()) family = proto.getFamily().toByteArray();
if (proto.hasQualifier()) qualifier = proto.getQualifier().toByteArray();
@@ -1603,7 +1605,7 @@
if (perm instanceof TablePermission) {
TablePermission tablePerm = (TablePermission)perm;
if (tablePerm.hasTable()) {
- builder.setTable(ByteString.copyFrom(tablePerm.getTable()));
+ builder.setTableName(ProtobufUtil.toProtoTableName(tablePerm.getTable()));
}
if (tablePerm.hasFamily()) {
builder.setFamily(ByteString.copyFrom(tablePerm.getFamily()));
@@ -1692,7 +1694,7 @@
permissionBuilder.addAction(toPermissionAction(a));
}
if (perm.hasTable()) {
- permissionBuilder.setTable(ByteString.copyFrom(perm.getTable()));
+ permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(perm.getTable()));
}
if (perm.hasFamily()) {
permissionBuilder.setFamily(ByteString.copyFrom(perm.getFamily()));
@@ -1719,9 +1721,9 @@
byte[] qualifier = null;
byte[] family = null;
- byte[] table = null;
+ TableName table = null;
- if (permission.hasTable()) table = permission.getTable().toByteArray();
+ if (permission.hasTableName()) table = ProtobufUtil.toTableName(permission.getTableName());
if (permission.hasFamily()) family = permission.getFamily().toByteArray();
if (permission.hasQualifier()) qualifier = permission.getQualifier().toByteArray();
@@ -1762,14 +1764,14 @@
*
* @param protocol the AccessControlService protocol proxy
* @param userShortName the short name of the user to grant permissions
- * @param t optional table name
+ * @param tableName optional table name
* @param f optional column family
* @param q optional qualifier
* @param actions the permissions to be granted
* @throws ServiceException
*/
public static void grant(AccessControlService.BlockingInterface protocol,
- String userShortName, byte[] t, byte[] f, byte[] q,
+ String userShortName, TableName tableName, byte[] f, byte[] q,
Permission.Action... actions) throws ServiceException {
List permActions =
Lists.newArrayListWithCapacity(actions.length);
@@ -1777,7 +1779,7 @@
permActions.add(ProtobufUtil.toPermissionAction(a));
}
AccessControlProtos.GrantRequest request = RequestConverter.
- buildGrantRequest(userShortName, t, f, q, permActions.toArray(
+ buildGrantRequest(userShortName, tableName, f, q, permActions.toArray(
new AccessControlProtos.Permission.Action[actions.length]));
protocol.grant(null, request);
}
@@ -1791,14 +1793,14 @@
*
* @param protocol the AccessControlService protocol proxy
* @param userShortName the short name of the user to revoke permissions
- * @param t optional table name
+ * @param tableName optional table name
* @param f optional column family
* @param q optional qualifier
* @param actions the permissions to be revoked
* @throws ServiceException
*/
public static void revoke(AccessControlService.BlockingInterface protocol,
- String userShortName, byte[] t, byte[] f, byte[] q,
+ String userShortName, TableName tableName, byte[] f, byte[] q,
Permission.Action... actions) throws ServiceException {
List permActions =
Lists.newArrayListWithCapacity(actions.length);
@@ -1806,7 +1808,7 @@
permActions.add(ProtobufUtil.toPermissionAction(a));
}
AccessControlProtos.RevokeRequest request = RequestConverter.
- buildRevokeRequest(userShortName, t, f, q, permActions.toArray(
+ buildRevokeRequest(userShortName, tableName, f, q, permActions.toArray(
new AccessControlProtos.Permission.Action[actions.length]));
protocol.revoke(null, request);
}
@@ -1822,11 +1824,11 @@
*/
public static List getUserPermissions(
AccessControlService.BlockingInterface protocol,
- byte[] t) throws ServiceException {
+ TableName t) throws ServiceException {
AccessControlProtos.UserPermissionsRequest.Builder builder =
AccessControlProtos.UserPermissionsRequest.newBuilder();
if (t != null) {
- builder.setTable(ByteString.copyFrom(t));
+ builder.setTableName(ProtobufUtil.toProtoTableName(t));
}
AccessControlProtos.UserPermissionsRequest request = builder.build();
AccessControlProtos.UserPermissionsResponse response =
@@ -1988,6 +1990,28 @@
cell.getValue().toByteArray());
}
+ public static HBaseProtos.NamespaceDescriptor toProtoNamespaceDescriptor(NamespaceDescriptor ns) {
+ HBaseProtos.NamespaceDescriptor.Builder b =
+ HBaseProtos.NamespaceDescriptor.newBuilder()
+ .setName(ByteString.copyFromUtf8(ns.getName()));
+ for(Map.Entry entry: ns.getConfiguration().entrySet()) {
+ b.addConfiguration(HBaseProtos.NameStringPair.newBuilder()
+ .setName(entry.getKey())
+ .setValue(entry.getValue()));
+ }
+ return b.build();
+ }
+
+ public static NamespaceDescriptor toNamespaceDescriptor(
+ HBaseProtos.NamespaceDescriptor desc) throws IOException {
+ NamespaceDescriptor.Builder b =
+ NamespaceDescriptor.create(desc.getName().toStringUtf8());
+ for(HBaseProtos.NameStringPair prop : desc.getConfigurationList()) {
+ b.addConfiguration(prop.getName(), prop.getValue());
+ }
+ return b.build();
+ }
+
/**
* Get an instance of the argument type declared in a class's signature. The
* argument type is assumed to be a PB Message subclass, and the instance is
@@ -2029,7 +2053,7 @@
// input / output paths are relative to the store dir
// store dir is relative to region dir
CompactionDescriptor.Builder builder = CompactionDescriptor.newBuilder()
- .setTableName(ByteString.copyFrom(info.getTableName()))
+ .setTableName(ByteString.copyFrom(info.getTableName().getName()))
.setEncodedRegionName(ByteString.copyFrom(info.getEncodedNameAsBytes()))
.setFamilyName(ByteString.copyFrom(family))
.setStoreHomeDir(storeDir.getName()); //make relative
@@ -2077,4 +2101,15 @@
return "row=" + Bytes.toString(proto.getRow().toByteArray()) +
", type=" + proto.getMutateType().toString();
}
+
+ public static TableName toTableName(HBaseProtos.TableName tableNamePB) {
+ return TableName.valueOf(tableNamePB.getNamespace().toByteArray(),
+ tableNamePB.getQualifier().toByteArray());
+ }
+
+ public static HBaseProtos.TableName toProtoTableName(TableName tableName) {
+ return HBaseProtos.TableName.newBuilder()
+ .setNamespace(ByteString.copyFrom(tableName.getNamespace()))
+ .setQualifier(ByteString.copyFrom(tableName.getQualifier())).build();
+ }
}
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (working copy)
@@ -22,6 +22,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CellScannable;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
@@ -904,9 +905,9 @@
* @return an AddColumnRequest
*/
public static AddColumnRequest buildAddColumnRequest(
- final byte [] tableName, final HColumnDescriptor column) {
+ final TableName tableName, final HColumnDescriptor column) {
AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
- builder.setTableName(ByteString.copyFrom(tableName));
+ builder.setTableName(ProtobufUtil.toProtoTableName(tableName));
builder.setColumnFamilies(column.convert());
return builder.build();
}
@@ -919,9 +920,9 @@
* @return a DeleteColumnRequest
*/
public static DeleteColumnRequest buildDeleteColumnRequest(
- final byte [] tableName, final byte [] columnName) {
+ final TableName tableName, final byte [] columnName) {
DeleteColumnRequest.Builder builder = DeleteColumnRequest.newBuilder();
- builder.setTableName(ByteString.copyFrom(tableName));
+ builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
builder.setColumnName(ByteString.copyFrom(columnName));
return builder.build();
}
@@ -934,9 +935,9 @@
* @return an ModifyColumnRequest
*/
public static ModifyColumnRequest buildModifyColumnRequest(
- final byte [] tableName, final HColumnDescriptor column) {
+ final TableName tableName, final HColumnDescriptor column) {
ModifyColumnRequest.Builder builder = ModifyColumnRequest.newBuilder();
- builder.setTableName(ByteString.copyFrom(tableName));
+ builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
builder.setColumnFamilies(column.convert());
return builder.build();
}
@@ -1019,9 +1020,9 @@
* @param tableName
* @return a DeleteTableRequest
*/
- public static DeleteTableRequest buildDeleteTableRequest(final byte [] tableName) {
+ public static DeleteTableRequest buildDeleteTableRequest(final TableName tableName) {
DeleteTableRequest.Builder builder = DeleteTableRequest.newBuilder();
- builder.setTableName(ByteString.copyFrom(tableName));
+ builder.setTableName(ProtobufUtil.toProtoTableName(tableName));
return builder.build();
}
@@ -1031,9 +1032,9 @@
* @param tableName
* @return an EnableTableRequest
*/
- public static EnableTableRequest buildEnableTableRequest(final byte [] tableName) {
+ public static EnableTableRequest buildEnableTableRequest(final TableName tableName) {
EnableTableRequest.Builder builder = EnableTableRequest.newBuilder();
- builder.setTableName(ByteString.copyFrom(tableName));
+ builder.setTableName(ProtobufUtil.toProtoTableName(tableName));
return builder.build();
}
@@ -1043,9 +1044,9 @@
* @param tableName
* @return a DisableTableRequest
*/
- public static DisableTableRequest buildDisableTableRequest(final byte [] tableName) {
+ public static DisableTableRequest buildDisableTableRequest(final TableName tableName) {
DisableTableRequest.Builder builder = DisableTableRequest.newBuilder();
- builder.setTableName(ByteString.copyFrom(tableName));
+ builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
return builder.build();
}
@@ -1077,9 +1078,9 @@
* @return a ModifyTableRequest
*/
public static ModifyTableRequest buildModifyTableRequest(
- final byte [] table, final HTableDescriptor hTableDesc) {
+ final TableName tableName, final HTableDescriptor hTableDesc) {
ModifyTableRequest.Builder builder = ModifyTableRequest.newBuilder();
- builder.setTableName(ByteString.copyFrom(table));
+ builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
builder.setTableSchema(hTableDesc.convert());
return builder.build();
}
@@ -1091,9 +1092,9 @@
* @return a GetSchemaAlterStatusRequest
*/
public static GetSchemaAlterStatusRequest buildGetSchemaAlterStatusRequest(
- final byte [] tableName) {
+ final TableName tableName) {
GetSchemaAlterStatusRequest.Builder builder = GetSchemaAlterStatusRequest.newBuilder();
- builder.setTableName(ByteString.copyFrom(tableName));
+ builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
return builder.build();
}
@@ -1104,11 +1105,11 @@
* @return a GetTableDescriptorsRequest
*/
public static GetTableDescriptorsRequest buildGetTableDescriptorsRequest(
- final List tableNames) {
+ final List tableNames) {
GetTableDescriptorsRequest.Builder builder = GetTableDescriptorsRequest.newBuilder();
if (tableNames != null) {
- for (String str : tableNames) {
- builder.addTableNames(str);
+ for (TableName tableName : tableNames) {
+ builder.addTableNames(ProtobufUtil.toProtoTableName(tableName));
}
}
return builder.build();
@@ -1121,9 +1122,9 @@
* @return a GetTableDescriptorsRequest
*/
public static GetTableDescriptorsRequest buildGetTableDescriptorsRequest(
- final byte[] tableName) {
+ final TableName tableName) {
return GetTableDescriptorsRequest.newBuilder()
- .addTableNames(Bytes.toString(tableName))
+ .addTableNames(ProtobufUtil.toProtoTableName(tableName))
.build();
}
@@ -1204,22 +1205,22 @@
* Create a request to grant user permissions.
*
* @param username the short user name who to grant permissions
- * @param table optional table name the permissions apply
+ * @param tableName optional table name the permissions apply
* @param family optional column family
* @param qualifier optional qualifier
* @param actions the permissions to be granted
* @return A {@link AccessControlProtos} GrantRequest
*/
public static AccessControlProtos.GrantRequest buildGrantRequest(
- String username, byte[] table, byte[] family, byte[] qualifier,
+ String username, TableName tableName, byte[] family, byte[] qualifier,
AccessControlProtos.Permission.Action... actions) {
AccessControlProtos.Permission.Builder permissionBuilder =
AccessControlProtos.Permission.newBuilder();
for (AccessControlProtos.Permission.Action a : actions) {
permissionBuilder.addAction(a);
}
- if (table != null) {
- permissionBuilder.setTable(ByteString.copyFrom(table));
+ if (tableName != null) {
+ permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
}
if (family != null) {
permissionBuilder.setFamily(ByteString.copyFrom(family));
@@ -1240,22 +1241,22 @@
* Create a request to revoke user permissions.
*
* @param username the short user name whose permissions to be revoked
- * @param table optional table name the permissions apply
+ * @param tableName optional table name the permissions apply
* @param family optional column family
* @param qualifier optional qualifier
* @param actions the permissions to be revoked
* @return A {@link AccessControlProtos} RevokeRequest
*/
public static AccessControlProtos.RevokeRequest buildRevokeRequest(
- String username, byte[] table, byte[] family, byte[] qualifier,
+ String username, TableName tableName, byte[] family, byte[] qualifier,
AccessControlProtos.Permission.Action... actions) {
AccessControlProtos.Permission.Builder permissionBuilder =
AccessControlProtos.Permission.newBuilder();
for (AccessControlProtos.Permission.Action a : actions) {
permissionBuilder.addAction(a);
}
- if (table != null) {
- permissionBuilder.setTable(ByteString.copyFrom(table));
+ if (tableName != null) {
+ permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
}
if (family != null) {
permissionBuilder.setFamily(ByteString.copyFrom(family));
@@ -1290,4 +1291,4 @@
}
return builder.build();
}
-}
\ No newline at end of file
+}
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java (working copy)
@@ -20,6 +20,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
@@ -36,7 +37,7 @@
public class TablePermission extends Permission {
private static Log LOG = LogFactory.getLog(TablePermission.class);
- private byte[] table;
+ private TableName table;
private byte[] family;
private byte[] qualifier;
@@ -52,7 +53,7 @@
* @param family the family, can be null if a global permission on the table
* @param assigned the list of allowed actions
*/
- public TablePermission(byte[] table, byte[] family, Action... assigned) {
+ public TablePermission(TableName table, byte[] family, Action... assigned) {
this(table, family, null, assigned);
}
@@ -63,7 +64,7 @@
* @param family the family, can be null if a global permission on the table
* @param assigned the list of allowed actions
*/
- public TablePermission(byte[] table, byte[] family, byte[] qualifier,
+ public TablePermission(TableName table, byte[] family, byte[] qualifier,
Action... assigned) {
super(assigned);
this.table = table;
@@ -78,7 +79,7 @@
* @param family the family, can be null if a global permission on the table
* @param actionCodes the list of allowed action codes
*/
- public TablePermission(byte[] table, byte[] family, byte[] qualifier,
+ public TablePermission(TableName table, byte[] family, byte[] qualifier,
byte[] actionCodes) {
super(actionCodes);
this.table = table;
@@ -90,7 +91,7 @@
return table != null;
}
- public byte[] getTable() {
+ public TableName getTable() {
return table;
}
@@ -123,9 +124,9 @@
* @return true if the action within the given scope is allowed
* by this permission, false
*/
- public boolean implies(byte[] table, byte[] family, byte[] qualifier,
+ public boolean implies(TableName table, byte[] family, byte[] qualifier,
Action action) {
- if (!Bytes.equals(this.table, table)) {
+ if (!this.table.equals(table)) {
return false;
}
@@ -154,8 +155,8 @@
* @return true if the action is allowed over the given scope
* by this permission, otherwise false
*/
- public boolean implies(byte[] table, KeyValue kv, Action action) {
- if (!Bytes.equals(this.table, table)) {
+ public boolean implies(TableName table, KeyValue kv, Action action) {
+ if (!this.table.equals(table)) {
return false;
}
@@ -183,8 +184,8 @@
* column-qualifier specific permission, for example, implies() would still
* return false.
*/
- public boolean matchesFamily(byte[] table, byte[] family, Action action) {
- if (!Bytes.equals(this.table, table)) {
+ public boolean matchesFamily(TableName table, byte[] family, Action action) {
+ if (!this.table.equals(table)) {
return false;
}
@@ -208,7 +209,7 @@
* @return true if the table, family and qualifier match,
* otherwise false
*/
- public boolean matchesFamilyQualifier(byte[] table, byte[] family, byte[] qualifier,
+ public boolean matchesFamilyQualifier(TableName table, byte[] family, byte[] qualifier,
Action action) {
if (!matchesFamily(table, family, action)) {
return false;
@@ -229,7 +230,7 @@
}
TablePermission other = (TablePermission)obj;
- if (!(Bytes.equals(table, other.getTable()) &&
+ if (!(table.equals(other.getTable()) &&
((family == null && other.getFamily() == null) ||
Bytes.equals(family, other.getFamily())) &&
((qualifier == null && other.getQualifier() == null) ||
@@ -247,7 +248,7 @@
final int prime = 37;
int result = super.hashCode();
if (table != null) {
- result = prime * result + Bytes.hashCode(table);
+ result = prime * result + table.hashCode();
}
if (family != null) {
result = prime * result + Bytes.hashCode(family);
@@ -260,7 +261,7 @@
public String toString() {
StringBuilder str = new StringBuilder("[TablePermission: ")
- .append("table=").append(Bytes.toString(table))
+ .append("table=").append(table)
.append(", family=").append(Bytes.toString(family))
.append(", qualifier=").append(Bytes.toString(qualifier))
.append(", actions=");
@@ -282,7 +283,8 @@
@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
- table = Bytes.readByteArray(in);
+ byte[] tableBytes = Bytes.readByteArray(in);
+ table = TableName.valueOf(tableBytes);
if (in.readBoolean()) {
family = Bytes.readByteArray(in);
}
@@ -294,7 +296,7 @@
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
- Bytes.writeByteArray(out, table);
+ Bytes.writeByteArray(out, table.getName());
out.writeBoolean(family != null);
if (family != null) {
Bytes.writeByteArray(out, family);
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/UserPermission.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/UserPermission.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/UserPermission.java (working copy)
@@ -20,6 +20,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.util.Bytes;
import java.io.DataInput;
@@ -69,7 +70,7 @@
* table
* @param assigned the list of allowed actions
*/
- public UserPermission(byte[] user, byte[] table, byte[] family,
+ public UserPermission(byte[] user, TableName table, byte[] family,
Action... assigned) {
super(table, family, assigned);
this.user = user;
@@ -86,7 +87,7 @@
* over the entire column family
* @param assigned the list of allowed actions
*/
- public UserPermission(byte[] user, byte[] table, byte[] family,
+ public UserPermission(byte[] user, TableName table, byte[] family,
byte[] qualifier, Action... assigned) {
super(table, family, qualifier, assigned);
this.user = user;
@@ -103,7 +104,7 @@
* over the entire column family
* @param actionCodes the list of allowed action codes
*/
- public UserPermission(byte[] user, byte[] table, byte[] family,
+ public UserPermission(byte[] user, TableName table, byte[] family,
byte[] qualifier, byte[] actionCodes) {
super(table, family, qualifier, actionCodes);
this.user = user;
@@ -117,8 +118,8 @@
* Returns true if this permission describes a global user permission.
*/
public boolean isGlobal() {
- byte[] tableName = getTable();
- return(tableName == null || tableName.length == 0);
+ TableName tableName = getTable();
+ return(tableName == null);
}
@Override
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java (working copy)
@@ -20,7 +20,9 @@
package org.apache.hadoop.hbase.snapshot;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.util.Bytes;
@@ -38,15 +40,16 @@
*/
public static void assertSnapshotRequestIsValid(HBaseProtos.SnapshotDescription snapshot)
throws IllegalArgumentException {
- // FIXME these method names is really bad - trunk will probably change
- // .META. and -ROOT- snapshots are not allowed
- if (HTableDescriptor.isMetaTable(Bytes.toBytes(snapshot.getTable()))) {
- throw new IllegalArgumentException(".META. and -ROOT- snapshots are not allowed");
+ // make sure the snapshot name is valid
+ TableName.isLegalTableQualifierName(Bytes.toBytes(snapshot.getName()));
+ if(snapshot.hasTable()) {
+ // make sure the table name is valid, this will implicitly check validity
+ TableName tableName = TableName.valueOf(snapshot.getTable());
+
+ if (HTableDescriptor.isSystemTable(tableName)) {
+ throw new IllegalArgumentException("System table snapshots are not allowed");
+ }
}
- // make sure the snapshot name is valid
- HTableDescriptor.isLegalTableName(Bytes.toBytes(snapshot.getName()));
- // make sure the table name is valid
- HTableDescriptor.isLegalTableName(Bytes.toBytes(snapshot.getTable()));
}
/**
@@ -60,7 +63,8 @@
if (ssd == null) {
return null;
}
- return "{ ss=" + ssd.getName() + " table=" + ssd.getTable()
- + " type=" + ssd.getType() + " }";
+ return "{ ss=" + ssd.getName() +
+ " table=" + (ssd.hasTable()?TableName.valueOf(ssd.getTable()):"") +
+ " type=" + ssd.getType() + " }";
}
}
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/TablePartiallyOpenException.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/TablePartiallyOpenException.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/TablePartiallyOpenException.java (working copy)
@@ -19,6 +19,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.util.Bytes;
import java.io.IOException;
@@ -45,7 +46,14 @@
/**
* @param tableName Name of table that is partial open
*/
- public TablePartiallyOpenException(byte[] tableName) {
- this(Bytes.toString(tableName));
+ public TablePartiallyOpenException(TableName tableName) {
+ this(tableName.getNameAsString());
}
+
+ /**
+ * @param tableName Name of table that is partial open
+ */
+ public TablePartiallyOpenException(byte[] tableName) {
+ this(Bytes.toString(tableName));
+ }
}
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java (working copy)
@@ -21,6 +21,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.zookeeper.KeeperException;
@@ -56,8 +57,8 @@
* for every query. Synchronize access rather than use concurrent Map because
* synchronization needs to span query of zk.
*/
- private final Map cache =
- new HashMap();
+ private final Map cache =
+ new HashMap();
// TODO: Make it so always a table znode. Put table schema here as well as table state.
// Have watcher on table znode so all are notified of state or schema change.
@@ -78,8 +79,9 @@
List children = ZKUtil.listChildrenNoWatch(this.watcher, this.watcher.tableZNode);
if (children == null) return;
for (String child: children) {
- ZooKeeperProtos.Table.State state = ZKTableReadOnly.getTableState(this.watcher, child);
- if (state != null) this.cache.put(child, state);
+ TableName tableName = TableName.valueOf(child);
+ ZooKeeperProtos.Table.State state = ZKTableReadOnly.getTableState(this.watcher, tableName);
+ if (state != null) this.cache.put(tableName, state);
}
}
}
@@ -90,7 +92,7 @@
* @param tableName
* @throws KeeperException unexpected zookeeper exception
*/
- public void setDisabledTable(String tableName)
+ public void setDisabledTable(TableName tableName)
throws KeeperException {
synchronized (this.cache) {
if (!isDisablingOrDisabledTable(tableName)) {
@@ -107,7 +109,7 @@
* @param tableName
* @throws KeeperException unexpected zookeeper exception
*/
- public void setDisablingTable(final String tableName)
+ public void setDisablingTable(final TableName tableName)
throws KeeperException {
synchronized (this.cache) {
if (!isEnabledOrDisablingTable(tableName)) {
@@ -124,7 +126,7 @@
* @param tableName
* @throws KeeperException unexpected zookeeper exception
*/
- public void setEnablingTable(final String tableName)
+ public void setEnablingTable(final TableName tableName)
throws KeeperException {
synchronized (this.cache) {
if (!isDisabledOrEnablingTable(tableName)) {
@@ -142,7 +144,7 @@
* @return if the operation succeeds or not
* @throws KeeperException unexpected zookeeper exception
*/
- public boolean checkAndSetEnablingTable(final String tableName)
+ public boolean checkAndSetEnablingTable(final TableName tableName)
throws KeeperException {
synchronized (this.cache) {
if (isEnablingTable(tableName)) {
@@ -160,7 +162,7 @@
* @return if the operation succeeds or not
* @throws KeeperException unexpected zookeeper exception
*/
- public boolean checkDisabledAndSetEnablingTable(final String tableName)
+ public boolean checkDisabledAndSetEnablingTable(final TableName tableName)
throws KeeperException {
synchronized (this.cache) {
if (!isDisabledTable(tableName)) {
@@ -178,7 +180,7 @@
* @return if the operation succeeds or not
* @throws KeeperException unexpected zookeeper exception
*/
- public boolean checkEnabledAndSetDisablingTable(final String tableName)
+ public boolean checkEnabledAndSetDisablingTable(final TableName tableName)
throws KeeperException {
synchronized (this.cache) {
if (this.cache.get(tableName) != null && !isEnabledTable(tableName)) {
@@ -189,9 +191,9 @@
}
}
- private void setTableState(final String tableName, final ZooKeeperProtos.Table.State state)
+ private void setTableState(final TableName tableName, final ZooKeeperProtos.Table.State state)
throws KeeperException {
- String znode = ZKUtil.joinZNode(this.watcher.tableZNode, tableName);
+ String znode = ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString());
if (ZKUtil.checkExists(this.watcher, znode) == -1) {
ZKUtil.createAndFailSilent(this.watcher, znode);
}
@@ -204,41 +206,41 @@
}
}
- public boolean isDisabledTable(final String tableName) {
+ public boolean isDisabledTable(final TableName tableName) {
return isTableState(tableName, ZooKeeperProtos.Table.State.DISABLED);
}
- public boolean isDisablingTable(final String tableName) {
+ public boolean isDisablingTable(final TableName tableName) {
return isTableState(tableName, ZooKeeperProtos.Table.State.DISABLING);
}
- public boolean isEnablingTable(final String tableName) {
+ public boolean isEnablingTable(final TableName tableName) {
return isTableState(tableName, ZooKeeperProtos.Table.State.ENABLING);
}
- public boolean isEnabledTable(String tableName) {
+ public boolean isEnabledTable(TableName tableName) {
return isTableState(tableName, ZooKeeperProtos.Table.State.ENABLED);
}
- public boolean isDisablingOrDisabledTable(final String tableName) {
+ public boolean isDisablingOrDisabledTable(final TableName tableName) {
synchronized (this.cache) {
return isDisablingTable(tableName) || isDisabledTable(tableName);
}
}
- public boolean isEnabledOrDisablingTable(final String tableName) {
+ public boolean isEnabledOrDisablingTable(final TableName tableName) {
synchronized (this.cache) {
return isEnabledTable(tableName) || isDisablingTable(tableName);
}
}
- public boolean isDisabledOrEnablingTable(final String tableName) {
+ public boolean isDisabledOrEnablingTable(final TableName tableName) {
synchronized (this.cache) {
return isDisabledTable(tableName) || isEnablingTable(tableName);
}
}
- private boolean isTableState(final String tableName, final ZooKeeperProtos.Table.State state) {
+ private boolean isTableState(final TableName tableName, final ZooKeeperProtos.Table.State state) {
synchronized (this.cache) {
ZooKeeperProtos.Table.State currentState = this.cache.get(tableName);
return ZKTableReadOnly.isTableState(currentState, state);
@@ -251,7 +253,7 @@
* @param tableName
* @throws KeeperException unexpected zookeeper exception
*/
- public void setDeletedTable(final String tableName)
+ public void setDeletedTable(final TableName tableName)
throws KeeperException {
synchronized (this.cache) {
if (this.cache.remove(tableName) == null) {
@@ -259,7 +261,7 @@
"already deleted");
}
ZKUtil.deleteNodeFailSilent(this.watcher,
- ZKUtil.joinZNode(this.watcher.tableZNode, tableName));
+ ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString()));
}
}
@@ -270,7 +272,7 @@
* @param tableName
* @throws KeeperException
*/
- public void setEnabledTable(final String tableName) throws KeeperException {
+ public void setEnabledTable(final TableName tableName) throws KeeperException {
setTableState(tableName, ZooKeeperProtos.Table.State.ENABLED);
}
@@ -280,7 +282,7 @@
* @param tableName
* @return true if the table is present
*/
- public boolean isTablePresent(final String tableName) {
+ public boolean isTablePresent(final TableName tableName) {
synchronized (this.cache) {
ZooKeeperProtos.Table.State state = this.cache.get(tableName);
return !(state == null);
@@ -291,11 +293,11 @@
* Gets a list of all the tables set as disabled in zookeeper.
* @return Set of disabled tables, empty Set if none
*/
- public Set getDisabledTables() {
- Set disabledTables = new HashSet();
+ public Set getDisabledTables() {
+ Set disabledTables = new HashSet();
synchronized (this.cache) {
- Set tables = this.cache.keySet();
- for (String table: tables) {
+ Set tables = this.cache.keySet();
+ for (TableName table: tables) {
if (isDisabledTable(table)) disabledTables.add(table);
}
}
@@ -307,7 +309,7 @@
* @return Set of disabled tables, empty Set if none
* @throws KeeperException
*/
- public static Set getDisabledTables(ZooKeeperWatcher zkw)
+ public static Set getDisabledTables(ZooKeeperWatcher zkw)
throws KeeperException {
return getAllTables(zkw, ZooKeeperProtos.Table.State.DISABLED);
}
@@ -317,7 +319,7 @@
* @return Set of disabling tables, empty Set if none
* @throws KeeperException
*/
- public static Set getDisablingTables(ZooKeeperWatcher zkw)
+ public static Set getDisablingTables(ZooKeeperWatcher zkw)
throws KeeperException {
return getAllTables(zkw, ZooKeeperProtos.Table.State.DISABLING);
}
@@ -327,7 +329,7 @@
* @return Set of enabling tables, empty Set if none
* @throws KeeperException
*/
- public static Set getEnablingTables(ZooKeeperWatcher zkw)
+ public static Set getEnablingTables(ZooKeeperWatcher zkw)
throws KeeperException {
return getAllTables(zkw, ZooKeeperProtos.Table.State.ENABLING);
}
@@ -337,7 +339,7 @@
* @return Set of disabled tables, empty Set if none
* @throws KeeperException
*/
- public static Set getDisabledOrDisablingTables(ZooKeeperWatcher zkw)
+ public static Set getDisabledOrDisablingTables(ZooKeeperWatcher zkw)
throws KeeperException {
return getAllTables(zkw, ZooKeeperProtos.Table.State.DISABLED,
ZooKeeperProtos.Table.State.DISABLING);
@@ -352,14 +354,14 @@
* @param deleteZNode
* @throws KeeperException
*/
- public void removeEnablingTable(final String tableName, boolean deleteZNode)
+ public void removeEnablingTable(final TableName tableName, boolean deleteZNode)
throws KeeperException {
synchronized (this.cache) {
if (isEnablingTable(tableName)) {
this.cache.remove(tableName);
if (deleteZNode) {
ZKUtil.deleteNodeFailSilent(this.watcher,
- ZKUtil.joinZNode(this.watcher.tableZNode, tableName));
+ ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString()));
}
}
}
@@ -371,17 +373,18 @@
* @return Set of tables of specified states, empty Set if none
* @throws KeeperException
*/
- static Set getAllTables(final ZooKeeperWatcher zkw,
+ static Set getAllTables(final ZooKeeperWatcher zkw,
final ZooKeeperProtos.Table.State... states) throws KeeperException {
- Set allTables = new HashSet();
+ Set allTables = new HashSet();
List children =
ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
if(children == null) return allTables;
for (String child: children) {
- ZooKeeperProtos.Table.State state = ZKTableReadOnly.getTableState(zkw, child);
+ TableName tableName = TableName.valueOf(child);
+ ZooKeeperProtos.Table.State state = ZKTableReadOnly.getTableState(zkw, tableName);
for (ZooKeeperProtos.Table.State expectedState: states) {
if (state == expectedState) {
- allTables.add(child);
+ allTables.add(tableName);
break;
}
}
Index: hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableReadOnly.java
===================================================================
--- hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableReadOnly.java (revision 1511579)
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableReadOnly.java (working copy)
@@ -20,6 +20,7 @@
package org.apache.hadoop.hbase.zookeeper;
import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
@@ -50,7 +51,7 @@
* @throws KeeperException
*/
public static boolean isDisabledTable(final ZooKeeperWatcher zkw,
- final String tableName)
+ final TableName tableName)
throws KeeperException {
ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
return isTableState(ZooKeeperProtos.Table.State.DISABLED, state);
@@ -66,7 +67,7 @@
* @throws KeeperException
*/
public static boolean isEnabledTable(final ZooKeeperWatcher zkw,
- final String tableName)
+ final TableName tableName)
throws KeeperException {
return getTableState(zkw, tableName) == ZooKeeperProtos.Table.State.ENABLED;
}
@@ -82,7 +83,7 @@
* @throws KeeperException
*/
public static boolean isDisablingOrDisabledTable(final ZooKeeperWatcher zkw,
- final String tableName)
+ final TableName tableName)
throws KeeperException {
ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
return isTableState(ZooKeeperProtos.Table.State.DISABLING, state) ||
@@ -94,14 +95,16 @@
* @return Set of disabled tables, empty Set if none
* @throws KeeperException
*/
- public static Set getDisabledTables(ZooKeeperWatcher zkw)
+ public static Set getDisabledTables(ZooKeeperWatcher zkw)
throws KeeperException {
- Set disabledTables = new HashSet();
+ Set disabledTables = new HashSet();
List children =
ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
for (String child: children) {
- ZooKeeperProtos.Table.State state = getTableState(zkw, child);
- if (state == ZooKeeperProtos.Table.State.DISABLED) disabledTables.add(child);
+ TableName tableName =
+ TableName.valueOf(child);
+ ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
+ if (state == ZooKeeperProtos.Table.State.DISABLED) disabledTables.add(tableName);
}
return disabledTables;
}
@@ -111,16 +114,18 @@
* @return Set of disabled tables, empty Set if none
* @throws KeeperException
*/
- public static Set