You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2013/08/08 08:08:31 UTC

svn commit: r1511591 [12/23] - in /hbase/branches/0.95: hbase-client/src/main/java/org/apache/hadoop/hbase/ hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/jav...

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Thu Aug  8 06:08:23 2013
@@ -73,6 +73,7 @@ import org.apache.hadoop.hbase.HColumnDe
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.RegionTooBusyException;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.UnknownScannerException;
 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
@@ -90,8 +91,8 @@ import org.apache.hadoop.hbase.client.Is
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.client.RowMutations;
+import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
@@ -760,7 +761,7 @@ public class HRegion implements HeapSize
   public static HDFSBlocksDistribution computeHDFSBlocksDistribution(final Configuration conf,
       final HTableDescriptor tableDescriptor, final HRegionInfo regionInfo) throws IOException {
     HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
-    Path tablePath = FSUtils.getTablePath(FSUtils.getRootDir(conf), tableDescriptor.getName());
+    Path tablePath = FSUtils.getTableDir(FSUtils.getRootDir(conf), tableDescriptor.getTableName());
     FileSystem fs = tablePath.getFileSystem(conf);
 
     HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tablePath, regionInfo);
@@ -2078,7 +2079,7 @@ public class HRegion implements HeapSize
           lastIndexExclusive++;
           continue;
         }
-        
+
         // If we haven't got any rows in our batch, we should block to
         // get the next one.
         boolean shouldBlock = numReadyToWrite == 0;
@@ -2159,8 +2160,8 @@ public class HRegion implements HeapSize
 
       // calling the pre CP hook for batch mutation
       if (!isInReplay && coprocessorHost != null) {
-        MiniBatchOperationInProgress<Mutation> miniBatchOp = 
-          new MiniBatchOperationInProgress<Mutation>(batchOp.operations, 
+        MiniBatchOperationInProgress<Mutation> miniBatchOp =
+          new MiniBatchOperationInProgress<Mutation>(batchOp.operations,
           batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
         if (coprocessorHost.preBatchMutate(miniBatchOp)) return 0L;
       }
@@ -2220,7 +2221,7 @@ public class HRegion implements HeapSize
       // -------------------------
       Mutation mutation = batchOp.operations[firstIndex];
       if (walEdit.size() > 0) {
-        txid = this.log.appendNoSync(this.getRegionInfo(), this.htableDescriptor.getName(),
+        txid = this.log.appendNoSync(this.getRegionInfo(), this.htableDescriptor.getTableName(),
                walEdit, mutation.getClusterId(), now, this.htableDescriptor);
       }
 
@@ -2232,7 +2233,7 @@ public class HRegion implements HeapSize
         locked = false;
       }
       releaseRowLocks(acquiredRowLocks);
-      
+
       // -------------------------
       // STEP 7. Sync wal.
       // -------------------------
@@ -2242,8 +2243,8 @@ public class HRegion implements HeapSize
       walSyncSuccessful = true;
       // calling the post CP hook for batch mutation
       if (!isInReplay && coprocessorHost != null) {
-        MiniBatchOperationInProgress<Mutation> miniBatchOp = 
-          new MiniBatchOperationInProgress<Mutation>(batchOp.operations, 
+        MiniBatchOperationInProgress<Mutation> miniBatchOp =
+          new MiniBatchOperationInProgress<Mutation>(batchOp.operations,
           batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
         coprocessorHost.postBatchMutate(miniBatchOp);
       }
@@ -3210,7 +3211,7 @@ public class HRegion implements HeapSize
           }
         }
       }
-      
+
       // allocate new lock for this thread
       return rowLockContext.newLock();
     } finally {
@@ -3937,11 +3938,11 @@ public class HRegion implements HeapSize
                                       final HLog hlog,
                                       final boolean initialize, final boolean ignoreHLog)
       throws IOException {
-    LOG.info("creating HRegion " + info.getTableNameAsString()
+    LOG.info("creating HRegion " + info.getTableName().getNameAsString()
         + " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
-        " Table name == " + info.getTableNameAsString());
+        " Table name == " + info.getTableName().getNameAsString());
 
-    Path tableDir = HTableDescriptor.getTableDir(rootDir, info.getTableName());
+    Path tableDir = FSUtils.getTableDir(rootDir, info.getTableName());
     FileSystem fs = FileSystem.get(conf);
     HRegionFileSystem rfs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, info);
     HLog effectiveHLog = hlog;
@@ -4098,8 +4099,10 @@ public class HRegion implements HeapSize
       final RegionServerServices rsServices, final CancelableProgressable reporter)
       throws IOException {
     if (info == null) throw new NullPointerException("Passed region info is null");
-    LOG.info("Open " + info);
-    Path dir = HTableDescriptor.getTableDir(rootDir, info.getTableName());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Opening region: " + info);
+    }
+    Path dir = FSUtils.getTableDir(rootDir, info.getTableName());
     HRegion r = HRegion.newHRegion(dir, wal, fs, conf, info, htd, rsServices);
     return r.openHRegion(reporter);
   }
@@ -4228,7 +4231,7 @@ public class HRegion implements HeapSize
   @Deprecated
   public static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
     return new Path(
-      HTableDescriptor.getTableDir(rootdir, info.getTableName()),
+      FSUtils.getTableDir(rootdir, info.getTableName()),
                                    info.getEncodedName());
   }
 
@@ -4288,8 +4291,8 @@ public class HRegion implements HeapSize
    * @throws IOException
    */
   public static HRegion merge(final HRegion a, final HRegion b) throws IOException {
-    if (!a.getRegionInfo().getTableNameAsString().equals(
-        b.getRegionInfo().getTableNameAsString())) {
+    if (!a.getRegionInfo().getTableName().equals(
+        b.getRegionInfo().getTableName())) {
       throw new IOException("Regions do not belong to the same table");
     }
 
@@ -4545,7 +4548,7 @@ public class HRegion implements HeapSize
           // 7. Append no sync
           if (!walEdit.isEmpty()) {
             txid = this.log.appendNoSync(this.getRegionInfo(),
-                this.htableDescriptor.getName(), walEdit,
+                this.htableDescriptor.getTableName(), walEdit,
                 processor.getClusterId(), now, this.htableDescriptor);
           }
           // 8. Release region lock
@@ -4691,10 +4694,10 @@ public class HRegion implements HeapSize
           long now = EnvironmentEdgeManager.currentTimeMillis();
           // Process each family
           for (Map.Entry<byte[], List<? extends Cell>> family : append.getFamilyMap().entrySet()) {
-  
+
             Store store = stores.get(family.getKey());
             List<KeyValue> kvs = new ArrayList<KeyValue>(family.getValue().size());
-  
+
             Collections.sort((List<KeyValue>)family.getValue(), store.getComparator());
             // Get previous values for all columns in this family
             Get get = new Get(row);
@@ -4703,10 +4706,10 @@ public class HRegion implements HeapSize
               get.addColumn(family.getKey(), kv.getQualifier());
             }
             List<KeyValue> results = get(get, false);
-  
+
             // Iterate the input columns and update existing values if they were
             // found, otherwise add new column initialized to the append value
-  
+
             // Avoid as much copying as possible. Every byte is copied at most
             // once.
             // Would be nice if KeyValue had scatter/gather logic
@@ -4750,10 +4753,10 @@ public class HRegion implements HeapSize
               System.arraycopy(kv.getBuffer(), kv.getQualifierOffset(),
                   newKV.getBuffer(), newKV.getQualifierOffset(),
                   kv.getQualifierLength());
-  
+
               newKV.setMemstoreTS(w.getWriteNumber());
               kvs.add(newKV);
-  
+
               // Append update to WAL
               if (writeToWAL) {
                 if (walEdits == null) {
@@ -4762,23 +4765,23 @@ public class HRegion implements HeapSize
                 walEdits.add(newKV);
               }
             }
-  
+
             //store the kvs to the temporary memstore before writing HLog
             tempMemstore.put(store, kvs);
           }
-  
+
           // Actually write to WAL now
           if (writeToWAL) {
             // Using default cluster id, as this can only happen in the orginating
             // cluster. A slave cluster receives the final value (not the delta)
             // as a Put.
-            txid = this.log.appendNoSync(this.getRegionInfo(), this.htableDescriptor.getName(),
+            txid = this.log.appendNoSync(this.getRegionInfo(), this.htableDescriptor.getTableName(),
               walEdits, HConstants.DEFAULT_CLUSTER_ID, EnvironmentEdgeManager.currentTimeMillis(),
               this.htableDescriptor);
           } else {
             recordMutationWithoutWal(append.getFamilyMap());
           }
-  
+
           //Actually write to Memstore now
           for (Map.Entry<Store, List<KeyValue>> entry : tempMemstore.entrySet()) {
             Store store = entry.getKey();
@@ -4866,10 +4869,10 @@ public class HRegion implements HeapSize
           // Process each family
           for (Map.Entry<byte [], List<? extends Cell>> family:
               increment.getFamilyMap().entrySet()) {
-  
+
             Store store = stores.get(family.getKey());
             List<KeyValue> kvs = new ArrayList<KeyValue>(family.getValue().size());
-  
+
             // Get previous values for all columns in this family
             Get get = new Get(row);
             for (Cell cell: family.getValue()) {
@@ -4878,7 +4881,7 @@ public class HRegion implements HeapSize
             }
             get.setTimeRange(tr.getMin(), tr.getMax());
             List<KeyValue> results = get(get, false);
-  
+
             // Iterate the input columns and update existing values if they were
             // found, otherwise add new column initialized to the increment amount
             int idx = 0;
@@ -4897,13 +4900,13 @@ public class HRegion implements HeapSize
                 }
                 idx++;
               }
-  
+
               // Append new incremented KeyValue to list
               KeyValue newKV =
                 new KeyValue(row, family.getKey(), qualifier, now, Bytes.toBytes(amount));
               newKV.setMemstoreTS(w.getWriteNumber());
               kvs.add(newKV);
-  
+
               // Prepare WAL updates
               if (writeToWAL) {
                 if (walEdits == null) {
@@ -4912,17 +4915,17 @@ public class HRegion implements HeapSize
                 walEdits.add(newKV);
               }
             }
-  
+
             //store the kvs to the temporary memstore before writing HLog
             tempMemstore.put(store, kvs);
           }
-  
+
           // Actually write to WAL now
           if (writeToWAL) {
             // Using default cluster id, as this can only happen in the orginating
             // cluster. A slave cluster receives the final value (not the delta)
             // as a Put.
-            txid = this.log.appendNoSync(this.getRegionInfo(), this.htableDescriptor.getName(),
+            txid = this.log.appendNoSync(this.getRegionInfo(), this.htableDescriptor.getTableName(),
                 walEdits, HConstants.DEFAULT_CLUSTER_ID, EnvironmentEdgeManager.currentTimeMillis(),
                 this.htableDescriptor);
           } else {
@@ -5141,9 +5144,8 @@ public class HRegion implements HeapSize
       final boolean majorCompact)
   throws IOException {
     HRegion region = null;
-    String metaStr = Bytes.toString(HConstants.META_TABLE_NAME);
     // Currently expects tables have one region only.
-    if (p.getName().startsWith(metaStr)) {
+    if (FSUtils.getTableName(p).equals(TableName.META_TABLE_NAME)) {
       region = HRegion.newHRegion(p, log, fs, c,
         HRegionInfo.FIRST_META_REGIONINFO, HTableDescriptor.META_TABLEDESC, null);
     } else {
@@ -5211,7 +5213,8 @@ public class HRegion implements HeapSize
    */
   public byte[] checkSplit() {
     // Can't split META
-    if (this.getRegionInfo().isMetaTable()) {
+    if (this.getRegionInfo().isMetaTable() ||
+        TableName.NAMESPACE_TABLE_NAME.equals(this.getRegionInfo().getTableName())) {
       if (shouldForceSplit()) {
         LOG.warn("Cannot split meta region in HBase 0.20 and above");
       }
@@ -5514,7 +5517,7 @@ public class HRegion implements HeapSize
     final Configuration c = HBaseConfiguration.create();
     final FileSystem fs = FileSystem.get(c);
     final Path logdir = new Path(c.get("hbase.tmp.dir"));
-    final String logname = "hlog" + tableDir.getName()
+    final String logname = "hlog" + FSUtils.getTableName(tableDir)
       + EnvironmentEdgeManager.currentTimeMillis();
 
     final HLog log = HLogFactory.createHLog(fs, logdir, logname, c);
@@ -5593,7 +5596,7 @@ public class HRegion implements HeapSize
      */
     void failedBulkLoad(byte[] family, String srcPath) throws IOException;
   }
-  
+
   @VisibleForTesting class RowLockContext {
     private final HashedBytes row;
     private final CountDownLatch latch = new CountDownLatch(1);
@@ -5604,16 +5607,16 @@ public class HRegion implements HeapSize
       this.row = row;
       this.thread = Thread.currentThread();
     }
-    
+
     boolean ownedByCurrentThread() {
       return thread == Thread.currentThread();
     }
-    
+
     RowLock newLock() {
       lockCount++;
       return new RowLock(this);
     }
-    
+
     void releaseLock() {
       if (!ownedByCurrentThread()) {
         throw new IllegalArgumentException("Lock held by thread: " + thread
@@ -5631,7 +5634,7 @@ public class HRegion implements HeapSize
       }
     }
   }
-  
+
   /**
    * Row lock held by a given thread.
    * One thread may acquire multiple locks on the same row simultaneously.
@@ -5640,11 +5643,11 @@ public class HRegion implements HeapSize
   public class RowLock {
     @VisibleForTesting final RowLockContext context;
     private boolean released = false;
-    
+
     @VisibleForTesting RowLock(RowLockContext context) {
       this.context = context;
     }
-    
+
     /**
      * Release the given lock.  If there are no remaining locks held by the current thread
      * then unlock the row and allow other threads to acquire the lock.

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Thu Aug  8 06:08:23 2013
@@ -59,6 +59,8 @@ import org.apache.hadoop.hbase.CellScann
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.Chore;
+import org.apache.hadoop.hbase.DaemonThreadFactory;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
@@ -2308,13 +2310,13 @@ public class HRegionServer implements Cl
    * @param tableName
    * @return Online regions from <code>tableName</code>
    */
-   @Override
-  public List<HRegion> getOnlineRegions(byte[] tableName) {
+  @Override
+  public List<HRegion> getOnlineRegions(TableName tableName) {
      List<HRegion> tableRegions = new ArrayList<HRegion>();
      synchronized (this.onlineRegions) {
        for (HRegion region: this.onlineRegions.values()) {
          HRegionInfo regionInfo = region.getRegionInfo();
-         if(Bytes.equals(regionInfo.getTableName(), tableName)) {
+         if(regionInfo.getTableName().equals(tableName)) {
            tableRegions.add(region);
          }
        }
@@ -3443,7 +3445,8 @@ public class HRegionServer implements Cl
     requestCount.increment();
     OpenRegionResponse.Builder builder = OpenRegionResponse.newBuilder();
     final int regionCount = request.getOpenInfoCount();
-    final Map<String, HTableDescriptor> htds = new HashMap<String, HTableDescriptor>(regionCount);
+    final Map<TableName, HTableDescriptor> htds =
+        new HashMap<TableName, HTableDescriptor>(regionCount);
     final boolean isBulkAssign = regionCount > 1;
     for (RegionOpenInfo regionOpenInfo : request.getOpenInfoList()) {
       final HRegionInfo region = HRegionInfo.convert(regionOpenInfo.getRegion());
@@ -3485,10 +3488,10 @@ public class HRegionServer implements Cl
           }
         }
         LOG.info("Open " + region.getRegionNameAsString());
-        htd = htds.get(region.getTableNameAsString());
+        htd = htds.get(region.getTableName());
         if (htd == null) {
           htd = this.tableDescriptors.get(region.getTableName());
-          htds.put(region.getTableNameAsString(), htd);
+          htds.put(region.getTableName(), htd);
         }
 
         final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java Thu Aug  8 06:08:23 2013
@@ -48,6 +48,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CompoundConfiguration;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -277,8 +278,8 @@ public class HStore implements Store {
   }
 
   @Override
-  public String getTableName() {
-    return this.getRegionInfo().getTableNameAsString();
+  public TableName getTableName() {
+    return this.getRegionInfo().getTableName();
   }
 
   @Override

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.java Thu Aug  8 06:08:23 2013
@@ -23,9 +23,9 @@ import java.util.List;
 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.HTableDescriptor;
-import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Split size is the number of regions that are on this server that all are
@@ -105,13 +105,13 @@ extends ConstantSizeRegionSplitPolicy {
     RegionServerServices rss = this.region.getRegionServerServices();
     // Can be null in tests
     if (rss == null) return 0;
-    byte [] tablename = this.region.getTableDesc().getName();
+    TableName tablename = this.region.getTableDesc().getTableName();
     int tableRegionsCount = 0;
     try {
       List<HRegion> hri = rss.getOnlineRegions(tablename);
       tableRegionsCount = hri == null || hri.isEmpty()? 0: hri.size();
     } catch (IOException e) {
-      LOG.debug("Failed getOnlineRegions " + Bytes.toString(tablename), e);
+      LOG.debug("Failed getOnlineRegions " + tablename, e);
     }
     return tableRegionsCount;
   }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyPrefixRegionSplitPolicy.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyPrefixRegionSplitPolicy.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyPrefixRegionSplitPolicy.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyPrefixRegionSplitPolicy.java Thu Aug  8 06:08:23 2013
@@ -54,7 +54,7 @@ public class KeyPrefixRegionSplitPolicy 
         prefixLengthString = region.getTableDesc().getValue(PREFIX_LENGTH_KEY_DEPRECATED);
         if (prefixLengthString == null) {
           LOG.error(PREFIX_LENGTH_KEY + " not specified for table "
-              + region.getTableDesc().getNameAsString()
+              + region.getTableDesc().getTableName()
               + ". Using default RegionSplitPolicy");
           return;
         }
@@ -66,7 +66,7 @@ public class KeyPrefixRegionSplitPolicy 
       }
       if (prefixLength <= 0) {
         LOG.error("Invalid value for " + PREFIX_LENGTH_KEY + " for table "
-            + region.getTableDesc().getNameAsString() + ":"
+            + region.getTableDesc().getTableName() + ":"
             + prefixLengthString + ". Using default RegionSplitPolicy");
       }
     }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java Thu Aug  8 06:08:23 2013
@@ -57,7 +57,7 @@ public class MetricsRegionWrapperImpl im
     if (tableDesc == null) {
       return "";
     }
-    return tableDesc.getNameAsString();
+    return tableDesc.getTableName().getNameAsString();
   }
 
   @Override

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnlineRegions.java Thu Aug  8 06:08:23 2013
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 
@@ -62,5 +63,5 @@ interface OnlineRegions extends Server {
     * @return List of HRegion
     * @throws java.io.IOException
     */
-   List<HRegion> getOnlineRegions(byte[] tableName) throws IOException;
+   List<HRegion> getOnlineRegions(TableName tableName) throws IOException;
 }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java Thu Aug  8 06:08:23 2013
@@ -147,7 +147,7 @@ public class RegionCoprocessorHost
     loadSystemCoprocessors(conf, REGION_COPROCESSOR_CONF_KEY);
 
     // load system default cp's for user tables from configuration.
-    if (!HTableDescriptor.isMetaTable(region.getRegionInfo().getTableName())) {
+    if (!HTableDescriptor.isSystemTable(region.getRegionInfo().getTableName())) {
       loadSystemCoprocessors(conf, USER_REGION_COPROCESSOR_CONF_KEY);
     }
 
@@ -195,7 +195,7 @@ public class RegionCoprocessorHost
               configured.add(load(path, className, priority, conf));
             }
             LOG.info("Load coprocessor " + className + " from HTD of " +
-              Bytes.toString(region.getTableDesc().getName()) +
+              region.getTableDesc().getTableName().getNameAsString() +
                 " successfully.");
           } else {
             throw new RuntimeException("specification does not match pattern");

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java Thu Aug  8 06:08:23 2013
@@ -70,7 +70,7 @@ class RegionMergeRequest implements Runn
 
       //acquire a shared read lock on the table, so that table schema modifications
       //do not happen concurrently
-      tableLock = server.getTableLockManager().readLock(region_a.getTableDesc().getName()
+      tableLock = server.getTableLockManager().readLock(region_a.getTableDesc().getTableName()
           , "MERGE_REGIONS:" + region_a.getRegionNameAsString() + ", " + region_b.getRegionNameAsString());
       try {
         tableLock.acquire();

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransaction.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransaction.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransaction.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeTransaction.java Thu Aug  8 06:08:23 2013
@@ -163,8 +163,8 @@ public class RegionMergeTransaction {
    *         <code>false</code> if they are not (e.g. its already closed, etc.).
    */
   public boolean prepare(final RegionServerServices services) {
-    if (!region_a.getTableDesc().getNameAsString()
-        .equals(region_b.getTableDesc().getNameAsString())) {
+    if (!region_a.getTableDesc().getTableName()
+        .equals(region_b.getTableDesc().getTableName())) {
       LOG.info("Can't merge regions " + region_a + "," + region_b
           + " because they do not belong to the same table");
       return false;

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java Thu Aug  8 06:08:23 2013
@@ -121,7 +121,7 @@ public abstract class RegionSplitPolicy 
     } catch (Exception  e) {
       throw new IOException(
           "Unable to load configured region split policy '" +
-          className + "' for table '" + htd.getNameAsString() + "'",
+          className + "' for table '" + htd.getTableName() + "'",
           e);
     }
   }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java Thu Aug  8 06:08:23 2013
@@ -66,7 +66,7 @@ class SplitRequest implements Runnable {
 
       //acquire a shared read lock on the table, so that table schema modifications
       //do not happen concurrently
-      tableLock = server.getTableLockManager().readLock(parent.getTableDesc().getName()
+      tableLock = server.getTableLockManager().readLock(parent.getTableDesc().getTableName()
           , "SPLIT_REGION:" + parent.getRegionNameAsString());
       try {
         tableLock.acquire();

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java Thu Aug  8 06:08:23 2013
@@ -34,7 +34,6 @@ import java.util.concurrent.TimeUnit;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.RegionTransition;
 import org.apache.hadoop.hbase.Server;

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java Thu Aug  8 06:08:23 2013
@@ -27,6 +27,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
@@ -327,7 +328,7 @@ public interface Store extends HeapSize,
 
   String getColumnFamilyName();
 
-  String getTableName();
+  TableName getTableName();
 
   /*
    * @param o Observer who wants to know about changes in set of Readers

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java Thu Aug  8 06:08:23 2013
@@ -262,7 +262,7 @@ public class StoreFileInfo {
 
   /*
    * Return path to the file referred to by a Reference.  Presumes a directory
-   * hierarchy of <code>${hbase.rootdir}/tablename/regionname/familyname</code>.
+   * hierarchy of <code>${hbase.rootdir}/data/${namespace}/tablename/regionname/familyname</code>.
    * @param p Path to a Reference file.
    * @return Calculated path to parent region file.
    * @throws IllegalArgumentException when path regex fails to match.

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java Thu Aug  8 06:08:23 2013
@@ -35,6 +35,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DaemonThreadFactory;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
 import org.apache.hadoop.hbase.master.snapshot.MasterSnapshotVerifier;
@@ -44,12 +45,12 @@ import org.apache.hadoop.hbase.procedure
 import org.apache.hadoop.hbase.procedure.Subprocedure;
 import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
 import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
@@ -225,8 +226,7 @@ public class RegionServerSnapshotManager
    * @throws IOException
    */
   private List<HRegion> getRegionsToSnapshot(SnapshotDescription snapshot) throws IOException {
-    byte[] table = Bytes.toBytes(snapshot.getTable());
-    return rss.getOnlineRegions(table);
+    return rss.getOnlineRegions(TableName.valueOf(snapshot.getTable()));
   }
 
   /**

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java Thu Aug  8 06:08:23 2013
@@ -49,6 +49,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Syncable;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -821,20 +822,20 @@ class FSHLog implements HLog, Syncable {
    * @param clusterId
    * @return New log key.
    */
-  protected HLogKey makeKey(byte[] encodedRegionName, byte[] tableName, long seqnum,
+  protected HLogKey makeKey(byte[] encodedRegionName, TableName tableName, long seqnum,
       long now, UUID clusterId) {
     return new HLogKey(encodedRegionName, tableName, seqnum, now, clusterId);
   }
 
   @Override
-  public void append(HRegionInfo info, byte [] tableName, WALEdit edits,
+  public void append(HRegionInfo info, TableName tableName, WALEdit edits,
     final long now, HTableDescriptor htd)
   throws IOException {
     append(info, tableName, edits, now, htd, true);
   }
 
   @Override
-  public void append(HRegionInfo info, byte [] tableName, WALEdit edits,
+  public void append(HRegionInfo info, TableName tableName, WALEdit edits,
     final long now, HTableDescriptor htd, boolean isInMemstore) throws IOException {
     append(info, tableName, edits, HConstants.DEFAULT_CLUSTER_ID, now, htd, true, isInMemstore);
   }
@@ -866,7 +867,7 @@ class FSHLog implements HLog, Syncable {
    * @throws IOException
    */
   @SuppressWarnings("deprecation")
-  private long append(HRegionInfo info, byte [] tableName, WALEdit edits, UUID clusterId,
+  private long append(HRegionInfo info, TableName tableName, WALEdit edits, UUID clusterId,
       final long now, HTableDescriptor htd, boolean doSync, boolean isInMemstore)
     throws IOException {
       if (edits.isEmpty()) return this.unflushedEntries.get();
@@ -905,7 +906,7 @@ class FSHLog implements HLog, Syncable {
     }
 
   @Override
-  public long appendNoSync(HRegionInfo info, byte [] tableName, WALEdit edits,
+  public long appendNoSync(HRegionInfo info, TableName tableName, WALEdit edits,
     UUID clusterId, final long now, HTableDescriptor htd)
     throws IOException {
     return append(info, tableName, edits, clusterId, now, htd, false, true);

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java Thu Aug  8 06:08:23 2013
@@ -32,6 +32,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
@@ -263,12 +264,11 @@ public interface HLog {
   void closeAndDelete() throws IOException;
 
   /**
-   * Same as {@link #appendNoSync(HRegionInfo, byte[], WALEdit, UUID, long, HTableDescriptor)},
+   * Same as {@link #appendNoSync(HRegionInfo, TableName, WALEdit, UUID, long, HTableDescriptor)},
    * except it causes a sync on the log
    */
-  void append(
-    HRegionInfo info, byte[] tableName, WALEdit edits, final long now, HTableDescriptor htd
-  ) throws IOException;
+  public void append(HRegionInfo info, TableName tableName, WALEdit edits,
+      final long now, HTableDescriptor htd) throws IOException;
 
   /**
    * Append a set of edits to the log. Log edits are keyed by (encoded)
@@ -281,14 +281,8 @@ public interface HLog {
    * @param htd
    * @param isInMemstore Whether the record is in memstore. False for system records.
    */
-  void append(
-    HRegionInfo info,
-    byte[] tableName,
-    WALEdit edits,
-    final long now,
-    HTableDescriptor htd,
-    boolean isInMemstore
-  ) throws IOException;
+  public void append(HRegionInfo info, TableName tableName, WALEdit edits,
+      final long now, HTableDescriptor htd, boolean isInMemstore) throws IOException;
 
   /**
    * Append a set of edits to the log. Log edits are keyed by (encoded)
@@ -305,14 +299,8 @@ public interface HLog {
    * @return txid of this transaction
    * @throws IOException
    */
-  long appendNoSync(
-    HRegionInfo info,
-    byte[] tableName,
-    WALEdit edits,
-    UUID clusterId,
-    final long now,
-    HTableDescriptor htd
-  ) throws IOException;
+  public long appendNoSync(HRegionInfo info, TableName tableName, WALEdit edits,
+      UUID clusterId, final long now, HTableDescriptor htd) throws IOException;
 
   void hsync() throws IOException;
 
@@ -333,7 +321,7 @@ public interface HLog {
    * to flush memstore.
    *
    * We stash the oldest seqNum for the region, and let the the next edit inserted in this
-   * region be recorded in {@link #append(HRegionInfo, byte[], WALEdit, long, HTableDescriptor)}
+   * region be recorded in {@link #append(HRegionInfo, TableName, WALEdit, long, HTableDescriptor)}
    * as new oldest seqnum. In case of flush being aborted, we put the stashed value back;
    * in case of flush succeeding, the seqNum of that first edit after start becomes the
    * valid oldest seqNum for this region.

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java Thu Aug  8 06:08:23 2013
@@ -22,7 +22,6 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.EOFException;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.NavigableMap;
@@ -32,6 +31,7 @@ import java.util.UUID;
 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.HConstants;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FamilyScope;
@@ -97,7 +97,7 @@ public class HLogKey implements Writable
 
   //  The encoded region name.
   private byte [] encodedRegionName;
-  private byte [] tablename;
+  private TableName tablename;
   private long logSeqNum;
   // Time at which this edit was written.
   private long writeTime;
@@ -125,7 +125,7 @@ public class HLogKey implements Writable
    * @param now Time at which this edit was written.
    * @param clusterId of the cluster (used in Replication)
    */
-  public HLogKey(final byte [] encodedRegionName, final byte [] tablename,
+  public HLogKey(final byte [] encodedRegionName, final TableName tablename,
       long logSeqNum, final long now, UUID clusterId) {
     this.logSeqNum = logSeqNum;
     this.writeTime = now;
@@ -155,7 +155,7 @@ public class HLogKey implements Writable
   }
 
   /** @return table name */
-  public byte [] getTablename() {
+  public TableName getTablename() {
     return tablename;
   }
 
@@ -197,7 +197,7 @@ public class HLogKey implements Writable
 
   @Override
   public String toString() {
-    return Bytes.toString(tablename) + "/" + Bytes.toString(encodedRegionName) + "/" +
+    return tablename + "/" + Bytes.toString(encodedRegionName) + "/" +
       logSeqNum;
   }
 
@@ -210,7 +210,7 @@ public class HLogKey implements Writable
    */
   public Map<String, Object> toStringMap() {
     Map<String, Object> stringMap = new HashMap<String, Object>();
-    stringMap.put("table", Bytes.toStringBinary(tablename));
+    stringMap.put("table", tablename);
     stringMap.put("region", Bytes.toStringBinary(encodedRegionName));
     stringMap.put("sequence", logSeqNum);
     return stringMap;
@@ -262,10 +262,10 @@ public class HLogKey implements Writable
    * meant to be a general purpose setter - it's only used
    * to collapse references to conserve memory.
    */
-  void internTableName(byte []tablename) {
+  void internTableName(TableName tablename) {
     // We should not use this as a setter - only to swap
     // in a new reference to the same table name.
-    assert Bytes.equals(tablename, this.tablename);
+    assert tablename.equals(this.tablename);
     this.tablename = tablename;
   }
 
@@ -289,12 +289,12 @@ public class HLogKey implements Writable
     WritableUtils.writeVInt(out, VERSION.code);
     if (compressionContext == null) {
       Bytes.writeByteArray(out, this.encodedRegionName);
-      Bytes.writeByteArray(out, this.tablename);
+      Bytes.writeByteArray(out, this.tablename.getName());
     } else {
       Compressor.writeCompressed(this.encodedRegionName, 0,
           this.encodedRegionName.length, out,
           compressionContext.regionDict);
-      Compressor.writeCompressed(this.tablename, 0, this.tablename.length, out,
+      Compressor.writeCompressed(this.tablename.getName(), 0, this.tablename.getName().length, out,
           compressionContext.tableDict);
     }
     out.writeLong(this.logSeqNum);
@@ -334,10 +334,12 @@ public class HLogKey implements Writable
     if (compressionContext == null || !version.atLeast(Version.COMPRESSED)) {
       this.encodedRegionName = new byte[len];
       in.readFully(this.encodedRegionName);
-      this.tablename = Bytes.readByteArray(in);
+      byte[] tablenameBytes = Bytes.readByteArray(in);
+      this.tablename = TableName.valueOf(tablenameBytes);
     } else {
       this.encodedRegionName = Compressor.readCompressed(in, compressionContext.regionDict);
-      this.tablename = Compressor.readCompressed(in, compressionContext.tableDict);
+      byte[] tablenameBytes = Compressor.readCompressed(in, compressionContext.tableDict);
+      this.tablename =  TableName.valueOf(tablenameBytes);
     }
 
     this.logSeqNum = in.readLong();
@@ -362,11 +364,12 @@ public class HLogKey implements Writable
     WALKey.Builder builder = WALKey.newBuilder();
     if (compressionContext == null) {
       builder.setEncodedRegionName(ByteString.copyFrom(this.encodedRegionName));
-      builder.setTableName(ByteString.copyFrom(this.tablename));
+      builder.setTableName(ByteString.copyFrom(this.tablename.getName()));
     } else {
       builder.setEncodedRegionName(
           compressor.compress(this.encodedRegionName, compressionContext.regionDict));
-      builder.setTableName(compressor.compress(this.tablename, compressionContext.tableDict));
+      builder.setTableName(compressor.compress(this.tablename.getName(),
+          compressionContext.tableDict));
     }
     builder.setLogSequenceNumber(this.logSeqNum);
     builder.setWriteTime(writeTime);
@@ -391,11 +394,12 @@ public class HLogKey implements Writable
     if (this.compressionContext != null) {
       this.encodedRegionName = uncompressor.uncompress(
           walKey.getEncodedRegionName(), compressionContext.regionDict);
-      this.tablename = uncompressor.uncompress(
+      byte[] tablenameBytes = uncompressor.uncompress(
           walKey.getTableName(), compressionContext.tableDict);
+      this.tablename = TableName.valueOf(tablenameBytes);
     } else {
       this.encodedRegionName = walKey.getEncodedRegionName().toByteArray();
-      this.tablename = walKey.getTableName().toByteArray();
+      this.tablename = TableName.valueOf(walKey.getTableName().toByteArray());
     }
     this.clusterId = HConstants.DEFAULT_CLUSTER_ID;
     if (walKey.hasClusterId()) {

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java Thu Aug  8 06:08:23 2013
@@ -22,6 +22,8 @@ import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
 import java.text.ParseException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -35,6 +37,7 @@ import java.util.TreeSet;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.Future;
@@ -51,10 +54,10 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
 import org.apache.hadoop.hbase.ServerName;
@@ -114,7 +117,8 @@ public class HLogSplitter {
   OutputSink outputSink;
   EntryBuffers entryBuffers;
 
-  private Set<String> disablingOrDisabledTables = new HashSet<String>();
+  private Set<TableName> disablingOrDisabledTables =
+      new HashSet<TableName>();
   private ZooKeeperWatcher watcher;
 
   // If an exception is thrown by one of the other threads, it will be
@@ -449,7 +453,7 @@ public class HLogSplitter {
   static Path getRegionSplitEditsPath(final FileSystem fs,
       final Entry logEntry, final Path rootDir, boolean isCreate)
   throws IOException {
-    Path tableDir = HTableDescriptor.getTableDir(rootDir, logEntry.getKey().getTablename());
+    Path tableDir = FSUtils.getTableDir(rootDir, logEntry.getKey().getTablename());
     String encodedRegionName = Bytes.toString(logEntry.getKey().getEncodedRegionName());
     Path regiondir = HRegion.getRegionDir(tableDir, encodedRegionName);
     Path dir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir);
@@ -747,11 +751,11 @@ public class HLogSplitter {
   static class RegionEntryBuffer implements HeapSize {
     long heapInBuffer = 0;
     List<Entry> entryBuffer;
-    byte[] tableName;
+    TableName tableName;
     byte[] encodedRegionName;
 
-    RegionEntryBuffer(byte[] table, byte[] region) {
-      this.tableName = table;
+    RegionEntryBuffer(TableName tableName, byte[] region) {
+      this.tableName = tableName;
       this.encodedRegionName = region;
       this.entryBuffer = new LinkedList<Entry>();
     }
@@ -957,7 +961,7 @@ public class HLogSplitter {
     abstract int getNumberOfRecoveredRegions();
 
     /**
-     * @param entry A WAL Edit Entry
+     * @param buffer A WAL Edit Entry
      * @throws IOException
      */
     abstract void append(RegionEntryBuffer buffer) throws IOException;
@@ -1304,8 +1308,8 @@ public class HLogSplitter {
     private final Map<String, HRegionLocation> onlineRegions =
         new ConcurrentHashMap<String, HRegionLocation>();
 
-    private Map<byte[], HConnection> tableNameToHConnectionMap = Collections
-        .synchronizedMap(new TreeMap<byte[], HConnection>(Bytes.BYTES_COMPARATOR));
+    private Map<TableName, HConnection> tableNameToHConnectionMap = Collections
+        .synchronizedMap(new TreeMap<TableName, HConnection>());
     /**
      * Map key -> value layout
      * <servername>:<table name> -> Queue<Row>
@@ -1337,7 +1341,7 @@ public class HLogSplitter {
       }
 
       // check if current region in a disabling or disabled table
-      if (disablingOrDisabledTables.contains(Bytes.toString(buffer.tableName))) {
+      if (disablingOrDisabledTables.contains(buffer.tableName)) {
         // need fall back to old way
         logRecoveredEditsOutputSink.append(buffer);
         hasEditsInDisablingOrDisabledTables = true;
@@ -1387,11 +1391,11 @@ public class HLogSplitter {
      * @throws IOException
      */
     private void groupEditsByServer(List<Entry> entries) throws IOException {
-      Set<byte[]> nonExistentTables = null;
+      Set<TableName> nonExistentTables = null;
       Long cachedLastFlushedSequenceId = -1l;
       for (HLog.Entry entry : entries) {
         WALEdit edit = entry.getEdit();
-        byte[] table = entry.getKey().getTablename();
+        TableName table = entry.getKey().getTablename();
         String encodeRegionNameStr = Bytes.toString(entry.getKey().getEncodedRegionName());
         // skip edits of non-existent tables
         if (nonExistentTables != null && nonExistentTables.contains(table)) {
@@ -1439,11 +1443,11 @@ public class HLogSplitter {
                 encodeRegionNameStr);
             } catch (TableNotFoundException ex) {
               // table has been deleted so skip edits of the table
-              LOG.info("Table " + Bytes.toString(table)
+              LOG.info("Table " + table
                   + " doesn't exist. Skip log replay for region " + encodeRegionNameStr);
               lastFlushedSequenceIds.put(encodeRegionNameStr, Long.MAX_VALUE);
               if (nonExistentTables == null) {
-                nonExistentTables = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
+                nonExistentTables = new TreeSet<TableName>();
               }
               nonExistentTables.add(table);
               this.skippedEdits.incrementAndGet();
@@ -1482,7 +1486,7 @@ public class HLogSplitter {
               put.setClusterId(entry.getKey().getClusterId());
               preRow = put;
             }
-            preKey = loc.getHostnamePort() + KEY_DELIMITER + Bytes.toString(table);
+            preKey = loc.getHostnamePort() + KEY_DELIMITER + table;
             preLoc = loc;
           }
           if (kv.isDelete()) {
@@ -1518,8 +1522,7 @@ public class HLogSplitter {
      * @throws IOException
      */
     private HRegionLocation locateRegionAndRefreshLastFlushedSequenceId(HConnection hconn,
-        byte[] table, byte[] row, String originalEncodedRegionName) throws IOException {
-
+        TableName table, byte[] row, String originalEncodedRegionName) throws IOException {
       // fetch location from cache
       HRegionLocation loc = onlineRegions.get(originalEncodedRegionName);
       if(loc != null) return loc;
@@ -1527,7 +1530,7 @@ public class HLogSplitter {
       loc = hconn.getRegionLocation(table, row, true);
       if (loc == null) {
         throw new IOException("Can't locate location for row:" + Bytes.toString(row)
-            + " of table:" + Bytes.toString(table));
+            + " of table:" + table);
       }
       // check if current row moves to a different region due to region merge/split
       if (!originalEncodedRegionName.equalsIgnoreCase(loc.getRegionInfo().getEncodedName())) {
@@ -1615,7 +1618,7 @@ public class HLogSplitter {
       final long pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
         HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
       boolean reloadLocation = false;
-      byte[] tableName = loc.getRegionInfo().getTableName();
+      TableName tableName = loc.getRegionInfo().getTableName();
       int tries = 0;
       Throwable cause = null;
       while (endTime > EnvironmentEdgeManager.currentTimeMillis()) {
@@ -1741,7 +1744,7 @@ public class HLogSplitter {
 
           // close connections
           synchronized (this.tableNameToHConnectionMap) {
-            for (byte[] tableName : this.tableNameToHConnectionMap.keySet()) {
+            for (TableName tableName : this.tableNameToHConnectionMap.keySet()) {
               HConnection hconn = this.tableNameToHConnectionMap.get(tableName);
               try {
                 hconn.clearRegionCache();
@@ -1783,23 +1786,23 @@ public class HLogSplitter {
         return ret;
       }
 
-      String tableName = getTableFromLocationStr(loc);
-      if(tableName.isEmpty()){
+      TableName tableName = getTableFromLocationStr(loc);
+      if(tableName != null){
         LOG.warn("Invalid location string:" + loc + " found.");
       }
 
-      HConnection hconn = getConnectionByTableName(Bytes.toBytes(tableName));
+      HConnection hconn = getConnectionByTableName(tableName);
       synchronized (writers) {
         ret = writers.get(loc);
         if (ret == null) {
-          ret = new RegionServerWriter(conf, Bytes.toBytes(tableName), hconn);
+          ret = new RegionServerWriter(conf, tableName, hconn);
           writers.put(loc, ret);
         }
       }
       return ret;
     }
 
-    private HConnection getConnectionByTableName(final byte[] tableName) throws IOException {
+    private HConnection getConnectionByTableName(final TableName tableName) throws IOException {
       HConnection hconn = this.tableNameToHConnectionMap.get(tableName);
       if (hconn == null) {
         synchronized (this.tableNameToHConnectionMap) {
@@ -1812,16 +1815,15 @@ public class HLogSplitter {
       }
       return hconn;
     }
-
-    private String getTableFromLocationStr(String loc) {
+    private TableName getTableFromLocationStr(String loc) {
       /**
        * location key is in format <server name:port>#<table name>
        */
       String[] splits = loc.split(KEY_DELIMITER);
       if (splits.length != 2) {
-        return "";
+        return null;
       }
-      return splits[1];
+      return TableName.valueOf(splits[1]);
     }
   }
 
@@ -1832,7 +1834,7 @@ public class HLogSplitter {
   private final static class RegionServerWriter extends SinkWriter {
     final WALEditsReplaySink sink;
 
-    RegionServerWriter(final Configuration conf, final byte[] tableName, final HConnection conn)
+    RegionServerWriter(final Configuration conf, final TableName tableName, final HConnection conn)
         throws IOException {
       this.sink = new WALEditsReplaySink(conf, tableName, conn);
     }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java Thu Aug  8 06:08:23 2013
@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -259,7 +260,7 @@ public class HLogUtil {
   public static void writeCompactionMarker(HLog log, HTableDescriptor htd, HRegionInfo info,
       final CompactionDescriptor c) throws IOException {
     WALEdit e = WALEdit.createCompaction(c);
-    log.append(info, c.getTableName().toByteArray(), e,
+    log.append(info, TableName.valueOf(c.getTableName().toByteArray()), e,
         EnvironmentEdgeManager.currentTimeMillis(), htd, false);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended compaction marker " + TextFormat.shortDebugString(c));

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java Thu Aug  8 06:08:23 2013
@@ -29,6 +29,7 @@ import org.apache.commons.logging.Log;
 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.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -65,7 +66,7 @@ public class WALEditsReplaySink {
 
   private final Configuration conf;
   private final HConnection conn;
-  private final byte[] tableName;
+  private final TableName tableName;
   private final MetricsWALEditsReplay metrics;
   private final AtomicLong totalReplayedEdits = new AtomicLong();
   private final boolean skipErrors;
@@ -78,7 +79,7 @@ public class WALEditsReplaySink {
    * @param conn
    * @throws IOException
    */
-  public WALEditsReplaySink(Configuration conf, byte[] tableName, HConnection conn)
+  public WALEditsReplaySink(Configuration conf, TableName tableName, HConnection conn)
       throws IOException {
     this.conf = conf;
     this.metrics = new MetricsWALEditsReplay();
@@ -185,7 +186,7 @@ public class WALEditsReplaySink {
     private HRegionInfo regionInfo;
     private List<Action<Row>> actions;
 
-    ReplayServerCallable(final HConnection connection, final byte [] tableName, 
+    ReplayServerCallable(final HConnection connection, final TableName tableName,
         final HRegionLocation regionLoc, final HRegionInfo regionInfo,
         final List<Action<Row>> actions) {
       super(connection, tableName, null);

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java Thu Aug  8 06:08:23 2013
@@ -37,6 +37,7 @@ import org.apache.hadoop.conf.Configurat
 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.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValueUtil;
@@ -51,7 +52,6 @@ import org.apache.hadoop.hbase.client.Pu
 import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
 
 /**
@@ -127,9 +127,11 @@ public class ReplicationSink {
       long totalReplicated = 0;
       // Map of table => list of Rows, we only want to flushCommits once per
       // invocation of this method per table.
-      Map<byte[], List<Row>> rows = new TreeMap<byte[], List<Row>>(Bytes.BYTES_COMPARATOR);
+      Map<TableName, List<Row>> rows =
+          new TreeMap<TableName, List<Row>>();
       for (WALEntry entry : entries) {
-        byte[] table = entry.getKey().getTableName().toByteArray();
+        TableName table =
+            TableName.valueOf(entry.getKey().getTableName().toByteArray());
         Cell previousCell = null;
         Mutation m = null;
         java.util.UUID uuid = toUUID(entry.getKey().getClusterId());
@@ -157,7 +159,7 @@ public class ReplicationSink {
         }
         totalReplicated++;
       }
-      for (Entry<byte[], List<Row>> entry : rows.entrySet()) {
+      for (Entry<TableName, List<Row>> entry : rows.entrySet()) {
         batch(entry.getKey(), entry.getValue());
       }
       int size = entries.size();
@@ -229,7 +231,7 @@ public class ReplicationSink {
    * @param rows list of actions
    * @throws IOException
    */
-  private void batch(byte[] tableName, List<Row> rows) throws IOException {
+  private void batch(TableName tableName, List<Row> rows) throws IOException {
     if (rows.isEmpty()) {
       return;
     }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java Thu Aug  8 06:08:23 2013
@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HConnectionManager;
@@ -398,8 +399,9 @@ public class ReplicationSource extends T
         removeNonReplicableEdits(entry);
         // Don't replicate catalog entries, if the WALEdit wasn't
         // containing anything to replicate and if we're currently not set to replicate
-        if (!(Bytes.equals(logKey.getTablename(), HConstants.ROOT_TABLE_NAME) || Bytes.equals(
-          logKey.getTablename(), HConstants.META_TABLE_NAME)) && edit.size() != 0) {
+        if (!(logKey.getTablename().equals(TableName.ROOT_TABLE_NAME) ||
+            logKey.getTablename().equals(TableName.META_TABLE_NAME)) &&
+            edit.size() != 0) {
           // Only set the clusterId if is a local key.
           // This ensures that the originator sets the cluster id
           // and all replicas retain the initial cluster id.

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java Thu Aug  8 06:08:23 2013
@@ -34,13 +34,13 @@ 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.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.MetaScanner;
 import org.apache.hadoop.hbase.rest.model.TableInfoModel;
 import org.apache.hadoop.hbase.rest.model.TableRegionModel;
-import org.apache.hadoop.hbase.util.Bytes;
 
 @InterfaceAudience.Private
 public class RegionsResource extends ResourceBase {
@@ -74,15 +74,15 @@ public class RegionsResource extends Res
     }
     servlet.getMetrics().incrementRequests(1);
     try {
-      String tableName = tableResource.getName();
-      TableInfoModel model = new TableInfoModel(tableName);
+      TableName tableName = TableName.valueOf(tableResource.getName());
+      TableInfoModel model = new TableInfoModel(tableName.getNameAsString());
       Map<HRegionInfo,ServerName> regions = MetaScanner.allTableRegions(
-        servlet.getConfiguration(), null, Bytes.toBytes(tableName), false);
+        servlet.getConfiguration(), null, tableName, false);
       for (Map.Entry<HRegionInfo,ServerName> e: regions.entrySet()) {
         HRegionInfo hri = e.getKey();
         ServerName addr = e.getValue();
         model.add(
-          new TableRegionModel(tableName, hri.getRegionId(),
+          new TableRegionModel(tableName.getNameAsString(), hri.getRegionId(),
             hri.getStartKey(), hri.getEndKey(), addr.getHostAndPort()));
       }
       ResponseBuilder response = Response.ok(model);

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java Thu Aug  8 06:08:23 2013
@@ -63,7 +63,7 @@ public class RootResource extends Resour
     TableListModel tableList = new TableListModel();
     HTableDescriptor[] list = servlet.getAdmin().listTables();
     for (HTableDescriptor htd: list) {
-      tableList.add(new TableModel(htd.getNameAsString()));
+      tableList.add(new TableModel(htd.getTableName().getNameAsString()));
     }
     return tableList;
   }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java Thu Aug  8 06:08:23 2013
@@ -42,6 +42,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -120,7 +121,7 @@ public class SchemaResource extends Reso
         .build();
     }
     try {
-      HTableDescriptor htd = new HTableDescriptor(name);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
       for (Map.Entry<QName,Object> e: model.getAny().entrySet()) {
         htd.setValue(e.getKey().getLocalPart(), e.getValue().toString());
       }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java Thu Aug  8 06:08:23 2013
@@ -279,7 +279,7 @@ public class RemoteAdmin {
       path.append(accessToken);
       path.append('/');
     }
-    path.append(Bytes.toStringBinary(desc.getName()));
+    path.append(desc.getTableName());
     path.append('/');
     path.append("schema");
     int code = 0;

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java Thu Aug  8 06:08:23 2013
@@ -33,6 +33,7 @@ import com.google.protobuf.ServiceExcept
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.*;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
@@ -233,6 +234,11 @@ public class RemoteHTable implements HTa
     return name.clone();
   }
 
+  @Override
+  public TableName getName() {
+    return TableName.valueOf(name);
+  }
+
   public Configuration getConfiguration() {
     return conf;
   }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/TableRegionModel.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/TableRegionModel.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/TableRegionModel.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/TableRegionModel.java Thu Aug  8 06:08:23 2013
@@ -25,6 +25,7 @@ import javax.xml.bind.annotation.XmlAttr
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -95,9 +96,10 @@ public class TableRegionModel implements
   @XmlAttribute
   public String getName() {
     byte [] tableNameAsBytes = Bytes.toBytes(this.table);
-    byte [] nameAsBytes = HRegionInfo.createRegionName(tableNameAsBytes,
+    byte [] nameAsBytes = HRegionInfo.createRegionName(
+        TableName.valueOf(tableNameAsBytes),
       this.startKey, this.id,
-      !HTableDescriptor.isMetaTable(tableNameAsBytes));
+      !HTableDescriptor.isSystemTable(TableName.valueOf(tableNameAsBytes)));
     return Bytes.toString(nameAsBytes);
   }
 

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java Thu Aug  8 06:08:23 2013
@@ -37,6 +37,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
 import org.apache.hadoop.hbase.rest.protobuf.generated.ColumnSchemaMessage.ColumnSchema;
@@ -83,7 +84,7 @@ public class TableSchemaModel implements
    * @param htd the table descriptor
    */
   public TableSchemaModel(HTableDescriptor htd) {
-    setName(htd.getNameAsString());
+    setName(htd.getTableName().getNameAsString());
     for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
         htd.getValues().entrySet()) {
       addAttribute(Bytes.toString(e.getKey().get()), 
@@ -337,7 +338,7 @@ public class TableSchemaModel implements
    * @return a table descriptor
    */
   public HTableDescriptor getTableDescriptor() {
-    HTableDescriptor htd = new HTableDescriptor(getName());
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(getName()));
     for (Map.Entry<QName, Object> e: getAny().entrySet()) {
       htd.setValue(e.getKey().getLocalPart(), e.getValue().toString());
     }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java Thu Aug  8 06:08:23 2013
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hbase.security.access;
 
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.filter.FilterBase;
@@ -41,7 +42,7 @@ import org.apache.hadoop.hbase.security.
 class AccessControlFilter extends FilterBase {
 
   private TableAuthManager authManager;
-  private byte[] table;
+  private TableName table;
   private User user;
 
   /**
@@ -51,7 +52,7 @@ class AccessControlFilter extends Filter
   }
 
   AccessControlFilter(TableAuthManager mgr, User ugi,
-      byte[] tableName) {
+      TableName tableName) {
     authManager = mgr;
     table = tableName;
     user = ugi;