You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2015/02/22 21:56:04 UTC

[31/50] [abbrv] hbase git commit: HBASE-12035 Keep table state in META (Andrey Stepachev)

HBASE-12035 Keep table state in META (Andrey Stepachev)


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/fa852c4c
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/fa852c4c
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/fa852c4c

Branch: refs/heads/hbase-11339
Commit: fa852c4ce68999cbe30e11820cbfd099d1c5cfaa
Parents: ede20aa
Author: stack <st...@apache.org>
Authored: Tue Feb 10 14:20:47 2015 -0800
Committer: stack <st...@apache.org>
Committed: Tue Feb 10 14:20:47 2015 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/HTableDescriptor.java   |  11 +
 .../apache/hadoop/hbase/MetaTableAccessor.java  | 385 ++++++++++++++-----
 .../hadoop/hbase/client/ConnectionCallable.java |  56 +++
 .../hadoop/hbase/client/ConnectionManager.java  | 137 +++----
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  88 +++--
 .../apache/hadoop/hbase/client/TableState.java  |  52 ++-
 .../org/apache/hadoop/hbase/HConstants.java     |  14 +-
 .../hbase/protobuf/generated/HBaseProtos.java   | 130 +++----
 hbase-protocol/src/main/protobuf/HBase.proto    |   4 +-
 .../apache/hadoop/hbase/TableDescriptor.java    |  51 ++-
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |   4 +-
 .../hadoop/hbase/master/AssignmentManager.java  |  11 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  39 +-
 .../hadoop/hbase/master/LoadBalancer.java       |   2 +
 .../hadoop/hbase/master/MasterFileSystem.java   |   2 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   2 -
 .../hadoop/hbase/master/RegionStateStore.java   |   3 +-
 .../hadoop/hbase/master/RegionStates.java       |   7 +-
 .../SnapshotOfRegionAssignmentFromMeta.java     |   2 +-
 .../hadoop/hbase/master/TableStateManager.java  | 206 +++++-----
 .../master/handler/CreateTableHandler.java      |  15 +-
 .../master/handler/TruncateTableHandler.java    |  11 +-
 .../hbase/regionserver/HRegionServer.java       |   8 +-
 .../hadoop/hbase/snapshot/SnapshotManifest.java |   3 +-
 .../hadoop/hbase/util/FSTableDescriptors.java   |   9 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java | 141 ++++---
 .../hadoop/hbase/HBaseTestingUtility.java       |  42 +-
 .../hadoop/hbase/TestMetaTableAccessor.java     |   7 +-
 .../apache/hadoop/hbase/client/TestAdmin1.java  |  28 +-
 .../hbase/client/TestMetaWithReplicas.java      |  69 +++-
 .../hbase/client/TestReplicaWithCluster.java    |   6 +-
 .../master/TestAssignmentManagerOnCluster.java  |   5 +-
 .../hadoop/hbase/master/TestCatalogJanitor.java |   3 +-
 .../TestMasterOperationsForRegionReplicas.java  |   6 +-
 .../hadoop/hbase/master/TestRegionStates.java   |   4 +-
 .../security/access/TestAccessController.java   |   1 +
 .../hbase/util/TestCoprocessorScanPolicy.java   |   1 +
 .../hbase/util/TestFSTableDescriptors.java      |  14 +-
 .../apache/hadoop/hbase/util/TestHBaseFsck.java |  31 +-
 .../util/hbck/OfflineMetaRebuildTestCore.java   |  13 +-
 .../util/hbck/TestOfflineMetaRebuildBase.java   |  13 +-
 41 files changed, 1095 insertions(+), 541 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
index cc91aed..51352bb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -1332,6 +1332,17 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
               .setBloomFilterType(BloomType.NONE)
               // Enable cache of data blocks in L1 if more than one caching tier deployed:
               // e.g. if using CombinedBlockCache (BucketCache).
+              .setCacheDataInL1(true),
+          new HColumnDescriptor(HConstants.TABLE_FAMILY)
+              // Ten is arbitrary number.  Keep versions to help debugging.
+              .setMaxVersions(10)
+              .setInMemory(true)
+              .setBlocksize(8 * 1024)
+              .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+                  // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+              .setBloomFilterType(BloomType.NONE)
+                  // Enable cache of data blocks in L1 if more than one caching tier deployed:
+                  // e.g. if using CombinedBlockCache (BucketCache).
               .setCacheDataInL1(true)
       });
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index 8f3a20e..2e6723a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -17,9 +17,23 @@
  */
 package org.apache.hadoop.hbase;
 
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ServiceException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -38,6 +52,8 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
@@ -48,18 +64,6 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PairOfSameType;
 import org.apache.hadoop.hbase.util.Threads;
 
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
 /**
  * Read/write operations on region and assignment information store in
  * <code>hbase:meta</code>.
@@ -78,6 +82,11 @@ public class MetaTableAccessor {
    * HRI defined which is called default replica.
    *
    * Meta layout (as of 0.98 + HBASE-10070) is like:
+   *
+   * For each table there is single row in column family 'table' formatted:
+   * <tableName> including namespace and columns are:
+   * table: state             => contains table state
+   *
    * For each table range, there is a single row, formatted like:
    * <tableName>,<startKey>,<regionId>,<encodedRegionName>. This row corresponds to the regionName
    * of the default region replica.
@@ -120,6 +129,24 @@ public class MetaTableAccessor {
       META_REGION_PREFIX, 0, len);
   }
 
+
+  @InterfaceAudience.Private
+  public enum QueryType {
+    ALL(HConstants.TABLE_FAMILY, HConstants.CATALOG_FAMILY),
+    REGION(HConstants.CATALOG_FAMILY),
+    TABLE(HConstants.TABLE_FAMILY);
+
+    private final byte[][] families;
+
+    QueryType(byte[]... families) {
+      this.families = families;
+    }
+
+    byte[][] getFamilies() {
+      return this.families;
+    }
+  }
+
   /** The delimiter for meta columns for replicaIds > 0 */
   protected static final char META_REPLICA_ID_DELIMITER = '_';
 
@@ -131,40 +158,64 @@ public class MetaTableAccessor {
   // Reading operations //
   ////////////////////////
 
- /**
-   * Performs a full scan of a <code>hbase:meta</code> table.
-   * @return List of {@link org.apache.hadoop.hbase.client.Result}
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
    * @throws IOException
    */
-  public static List<Result> fullScanOfMeta(Connection connection)
-  throws IOException {
-    CollectAllVisitor v = new CollectAllVisitor();
-    fullScan(connection, v, null);
-    return v.getResults();
+  public static void fullScanRegions(Connection connection,
+      final Visitor visitor)
+      throws IOException {
+    fullScan(connection, visitor, null, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @throws IOException
+   */
+  public static List<Result> fullScanRegions(Connection connection)
+      throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   * @throws IOException
+   */
+  public static void fullScanTables(Connection connection,
+      final Visitor visitor)
+      throws IOException {
+    fullScan(connection, visitor, null, QueryType.TABLE);
   }
 
   /**
    * Performs a full scan of <code>hbase:meta</code>.
    * @param connection connection we're using
    * @param visitor Visitor invoked against each row.
+   * @param type scanned part of meta
    * @throws IOException
    */
   public static void fullScan(Connection connection,
-      final Visitor visitor)
+      final Visitor visitor, QueryType type)
   throws IOException {
-    fullScan(connection, visitor, null);
+    fullScan(connection, visitor, null, type);
   }
 
   /**
    * Performs a full scan of <code>hbase:meta</code>.
    * @param connection connection we're using
+   * @param type scanned part of meta
    * @return List of {@link Result}
    * @throws IOException
    */
-  public static List<Result> fullScan(Connection connection)
+  public static List<Result> fullScan(Connection connection, QueryType type)
     throws IOException {
     CollectAllVisitor v = new CollectAllVisitor();
-    fullScan(connection, v, null);
+    fullScan(connection, v, null, type);
     return v.getResults();
   }
 
@@ -306,6 +357,7 @@ public class MetaTableAccessor {
    * @return null if it doesn't contain merge qualifier, else two merge regions
    * @throws IOException
    */
+  @Nullable
   public static Pair<HRegionInfo, HRegionInfo> getRegionsFromMergeQualifier(
       Connection connection, byte[] regionName) throws IOException {
     Result result = getRegionResult(connection, regionName);
@@ -328,42 +380,9 @@ public class MetaTableAccessor {
   public static boolean tableExists(Connection connection,
       final TableName tableName)
   throws IOException {
-    if (tableName.equals(TableName.META_TABLE_NAME)) {
-      // Catalog tables always exist.
-      return true;
-    }
-    // Make a version of ResultCollectingVisitor that only collects the first
-    CollectingVisitor<HRegionInfo> visitor = new CollectingVisitor<HRegionInfo>() {
-      private HRegionInfo current = null;
-
-      @Override
-      public boolean visit(Result r) throws IOException {
-        RegionLocations locations = getRegionLocations(r);
-        if (locations == null || locations.getRegionLocation().getRegionInfo() == null) {
-          LOG.warn("No serialized HRegionInfo in " + r);
-          return true;
-        }
-        this.current = locations.getRegionLocation().getRegionInfo();
-        if (this.current == null) {
-          LOG.warn("No serialized HRegionInfo in " + r);
-          return true;
-        }
-        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.
-        return false;
-      }
-
-      @Override
-      void add(Result r) {
-        // Add the current HRI.
-        this.results.add(this.current);
-      }
-    };
-    fullScan(connection, visitor, getTableStartRowForMeta(tableName));
-    // If visitor has results >= 1 then table exists.
-    return visitor.getResults().size() >= 1;
+    // Catalog tables always exist.
+    return tableName.equals(TableName.META_TABLE_NAME)
+        || getTableState(connection, tableName) != null;
   }
 
   /**
@@ -400,6 +419,7 @@ public class MetaTableAccessor {
     return getListOfHRegionInfos(result);
   }
 
+  @Nullable
   static List<HRegionInfo> getListOfHRegionInfos(final List<Pair<HRegionInfo, ServerName>> pairs) {
     if (pairs == null || pairs.isEmpty()) return null;
     List<HRegionInfo> result = new ArrayList<HRegionInfo>(pairs.size());
@@ -470,6 +490,7 @@ public class MetaTableAccessor {
    * Do not use this method to get meta table regions, use methods in MetaTableLocator instead.
    * @param connection connection we're using
    * @param tableName table to work with
+   * @param excludeOfflinedSplitParents don't return split parents
    * @return Return list of regioninfos and server addresses.
    * @throws IOException
    */
@@ -512,7 +533,7 @@ public class MetaTableAccessor {
           }
         }
       };
-    fullScan(connection, visitor, getTableStartRowForMeta(tableName));
+    fullScan(connection, visitor, getTableStartRowForMeta(tableName), QueryType.REGION);
     return visitor.getResults();
   }
 
@@ -544,7 +565,7 @@ public class MetaTableAccessor {
         }
       }
     };
-    fullScan(connection, v);
+    fullScan(connection, v, QueryType.REGION);
     return hris;
   }
 
@@ -555,17 +576,22 @@ public class MetaTableAccessor {
       public boolean visit(Result r) throws IOException {
         if (r ==  null || r.isEmpty()) return true;
         LOG.info("fullScanMetaAndPrint.Current Meta Row: " + r);
-        RegionLocations locations = getRegionLocations(r);
-        if (locations == null) return true;
-        for (HRegionLocation loc : locations.getRegionLocations()) {
-          if (loc != null) {
-            LOG.info("fullScanMetaAndPrint.HRI Print= " + loc.getRegionInfo());
+        TableState state = getTableState(r);
+        if (state != null) {
+          LOG.info("Table State: " + state);
+        } else {
+          RegionLocations locations = getRegionLocations(r);
+          if (locations == null) return true;
+          for (HRegionLocation loc : locations.getRegionLocations()) {
+            if (loc != null) {
+              LOG.info("fullScanMetaAndPrint.HRI Print= " + loc.getRegionInfo());
+            }
           }
         }
         return true;
       }
     };
-    fullScan(connection, v);
+    fullScan(connection, v, QueryType.ALL);
   }
 
   /**
@@ -574,20 +600,40 @@ public class MetaTableAccessor {
    * @param visitor Visitor invoked against each row.
    * @param startrow Where to start the scan. Pass null if want to begin scan
    * at first row.
+   * @param type scanned part of meta
    * <code>hbase:meta</code>, the default (pass false to scan hbase:meta)
    * @throws IOException
    */
   public static void fullScan(Connection connection,
-    final Visitor visitor, final byte [] startrow)
+      final Visitor visitor, @Nullable final byte[] startrow, QueryType type) throws IOException {
+    fullScan(connection, visitor, startrow, type, false);
+  }
+
+  /**
+   * Performs a full scan of a catalog table.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row.
+   * @param startrow Where to start the scan. Pass null if want to begin scan
+   * at first row.
+   * @param type scanned part of meta
+   * @param raw read raw data including Delete tumbstones
+   * <code>hbase:meta</code>, the default (pass false to scan hbase:meta)
+   * @throws IOException
+   */
+  public static void fullScan(Connection connection,
+      final Visitor visitor, @Nullable final byte[] startrow, QueryType type, boolean raw)
   throws IOException {
     Scan scan = new Scan();
+    scan.setRaw(raw);
     if (startrow != null) scan.setStartRow(startrow);
     if (startrow == null) {
       int caching = connection.getConfiguration()
           .getInt(HConstants.HBASE_META_SCANNER_CACHING, 100);
       scan.setCaching(caching);
     }
-    scan.addFamily(HConstants.CATALOG_FAMILY);
+    for (byte[] family : type.getFamilies()) {
+      scan.addFamily(family);
+    }
     Table metaTable = getMetaHTable(connection);
     ResultScanner scanner = null;
     try {
@@ -608,11 +654,19 @@ public class MetaTableAccessor {
    * Returns the column family used for meta columns.
    * @return HConstants.CATALOG_FAMILY.
    */
-  protected static byte[] getFamily() {
+  protected static byte[] getCatalogFamily() {
     return HConstants.CATALOG_FAMILY;
   }
 
   /**
+   * Returns the column family used for table columns.
+   * @return HConstants.TABLE_FAMILY.
+   */
+  protected static byte[] getTableFamily() {
+    return HConstants.TABLE_FAMILY;
+  }
+
+  /**
    * Returns the column qualifier for serialized region info
    * @return HConstants.REGIONINFO_QUALIFIER
    */
@@ -621,6 +675,15 @@ public class MetaTableAccessor {
   }
 
   /**
+   * Returns the column qualifier for serialized table state
+   *
+   * @return HConstants.TABLE_STATE_QUALIFIER
+   */
+  protected static byte[] getStateColumn() {
+    return HConstants.TABLE_STATE_QUALIFIER;
+  }
+
+  /**
    * Returns the column qualifier for server column for replicaId
    * @param replicaId the replicaId of the region
    * @return a byte[] for server column qualifier
@@ -686,14 +749,15 @@ public class MetaTableAccessor {
    * @param r Result to pull from
    * @return A ServerName instance or null if necessary fields not found or empty.
    */
+  @Nullable
   private static ServerName getServerName(final Result r, final int replicaId) {
     byte[] serverColumn = getServerColumn(replicaId);
-    Cell cell = r.getColumnLatestCell(getFamily(), serverColumn);
+    Cell cell = r.getColumnLatestCell(getCatalogFamily(), serverColumn);
     if (cell == null || cell.getValueLength() == 0) return null;
     String hostAndPort = Bytes.toString(
       cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
     byte[] startcodeColumn = getStartCodeColumn(replicaId);
-    cell = r.getColumnLatestCell(getFamily(), startcodeColumn);
+    cell = r.getColumnLatestCell(getCatalogFamily(), startcodeColumn);
     if (cell == null || cell.getValueLength() == 0) return null;
     return ServerName.valueOf(hostAndPort,
       Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
@@ -706,7 +770,7 @@ public class MetaTableAccessor {
    * @return SeqNum, or HConstants.NO_SEQNUM if there's no value written.
    */
   private static long getSeqNumDuringOpen(final Result r, final int replicaId) {
-    Cell cell = r.getColumnLatestCell(getFamily(), getSeqNumColumn(replicaId));
+    Cell cell = r.getColumnLatestCell(getCatalogFamily(), getSeqNumColumn(replicaId));
     if (cell == null || cell.getValueLength() == 0) return HConstants.NO_SEQNUM;
     return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
   }
@@ -716,6 +780,7 @@ public class MetaTableAccessor {
    * @return an HRegionLocationList containing all locations for the region range or null if
    *  we can't deserialize the result.
    */
+  @Nullable
   public static RegionLocations getRegionLocations(final Result r) {
     if (r == null) return null;
     HRegionInfo regionInfo = getHRegionInfo(r, getRegionInfoColumn());
@@ -726,7 +791,7 @@ public class MetaTableAccessor {
 
     locations.add(getRegionLocation(r, regionInfo, 0));
 
-    NavigableMap<byte[], byte[]> infoMap = familyMap.get(getFamily());
+    NavigableMap<byte[], byte[]> infoMap = familyMap.get(getCatalogFamily());
     if (infoMap == null) return new RegionLocations(locations);
 
     // iterate until all serverName columns are seen
@@ -788,8 +853,9 @@ public class MetaTableAccessor {
    * @param qualifier Column family qualifier
    * @return An HRegionInfo instance or null.
    */
+  @Nullable
   private static HRegionInfo getHRegionInfo(final Result r, byte [] qualifier) {
-    Cell cell = r.getColumnLatestCell(getFamily(), qualifier);
+    Cell cell = r.getColumnLatestCell(getCatalogFamily(), qualifier);
     if (cell == null) return null;
     return HRegionInfo.parseFromOrNull(cell.getValueArray(),
       cell.getValueOffset(), cell.getValueLength());
@@ -824,6 +890,80 @@ public class MetaTableAccessor {
   }
 
   /**
+   * Fetch table state for given table from META table
+   * @param conn connection to use
+   * @param tableName table to fetch state for
+   * @return state
+   * @throws IOException
+   */
+  @Nullable
+  public static TableState getTableState(Connection conn, TableName tableName)
+      throws IOException {
+    Table metaHTable = getMetaHTable(conn);
+    Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getStateColumn());
+    long time = EnvironmentEdgeManager.currentTime();
+    get.setTimeRange(0, time);
+    Result result =
+        metaHTable.get(get);
+    return getTableState(result);
+  }
+
+  /**
+   * Fetch table states from META table
+   * @param conn connection to use
+   * @return map {tableName -> state}
+   * @throws IOException
+   */
+  public static Map<TableName, TableState> getTableStates(Connection conn)
+      throws IOException {
+    final Map<TableName, TableState> states = new LinkedHashMap<>();
+    Visitor collector = new Visitor() {
+      @Override
+      public boolean visit(Result r) throws IOException {
+        TableState state = getTableState(r);
+        if (state != null)
+          states.put(state.getTableName(), state);
+        return true;
+      }
+    };
+    fullScanTables(conn, collector);
+    return states;
+  }
+
+  /**
+   * Updates state in META
+   * @param conn connection to use
+   * @param tableName table to look for
+   * @throws IOException
+   */
+  public static void updateTableState(Connection conn, TableName tableName,
+      TableState.State actual) throws IOException {
+    updateTableState(conn, new TableState(tableName, actual));
+  }
+
+  /**
+   * Decode table state from META Result.
+   * Should contain cell from HConstants.TABLE_FAMILY
+   * @param r result
+   * @return null if not found
+   * @throws IOException
+   */
+  @Nullable
+  public static TableState getTableState(Result r)
+      throws IOException {
+    Cell cell = r.getColumnLatestCell(getTableFamily(), getStateColumn());
+    if (cell == null) return null;
+    try {
+      return TableState.parseFrom(TableName.valueOf(r.getRow()),
+          Arrays.copyOfRange(cell.getValueArray(),
+          cell.getValueOffset(), cell.getValueOffset() + cell.getValueLength()));
+    } catch (DeserializationException e) {
+      throw new IOException(e);
+    }
+
+  }
+
+  /**
    * Implementations 'visit' a catalog table row.
    */
   public interface Visitor {
@@ -920,7 +1060,8 @@ public class MetaTableAccessor {
    */
   public static Put makePutFromRegionInfo(HRegionInfo regionInfo)
     throws IOException {
-    Put put = new Put(regionInfo.getRegionName());
+    long now = EnvironmentEdgeManager.currentTime();
+    Put put = new Put(regionInfo.getRegionName(), now);
     addRegionInfo(put, regionInfo);
     return put;
   }
@@ -933,7 +1074,9 @@ public class MetaTableAccessor {
     if (regionInfo == null) {
       throw new IllegalArgumentException("Can't make a delete for null region");
     }
+    long now = EnvironmentEdgeManager.currentTime();
     Delete delete = new Delete(regionInfo.getRegionName());
+    delete.addFamily(getCatalogFamily(), now);
     return delete;
   }
 
@@ -1034,14 +1177,15 @@ public class MetaTableAccessor {
       throws IOException {
     int absoluteIndex = replicaIndexToDeleteFrom + numReplicasToRemove;
     for (byte[] row : metaRows) {
+      long now = EnvironmentEdgeManager.currentTime();
       Delete deleteReplicaLocations = new Delete(row);
       for (int i = replicaIndexToDeleteFrom; i < absoluteIndex; i++) {
-        deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY,
-          getServerColumn(i));
-        deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY,
-          getSeqNumColumn(i));
-        deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY,
-          getStartCodeColumn(i));
+        deleteReplicaLocations.addColumns(getCatalogFamily(),
+          getServerColumn(i), now);
+        deleteReplicaLocations.addColumns(getCatalogFamily(),
+          getSeqNumColumn(i), now);
+        deleteReplicaLocations.addColumns(getCatalogFamily(),
+          getStartCodeColumn(i), now);
       }
       deleteFromMetaTable(connection, deleteReplicaLocations);
     }
@@ -1171,7 +1315,8 @@ public class MetaTableAccessor {
   public static void addDaughter(final Connection connection,
       final HRegionInfo regionInfo, final ServerName sn, final long openSeqNum)
       throws NotAllMetaRegionsOnlineException, IOException {
-    Put put = new Put(regionInfo.getRegionName());
+    long now = EnvironmentEdgeManager.currentTime();
+    Put put = new Put(regionInfo.getRegionName(), now);
     addRegionInfo(put, regionInfo);
     if (sn != null) {
       addLocation(put, sn, openSeqNum, regionInfo.getReplicaId());
@@ -1273,6 +1418,45 @@ public class MetaTableAccessor {
   }
 
   /**
+   * Update state of the table in meta.
+   * @param connection what we use for update
+   * @param state new state
+   * @throws IOException
+   */
+  public static void updateTableState(Connection connection, TableState state)
+      throws IOException {
+    Put put = makePutFromTableState(state);
+    putToMetaTable(connection, put);
+    LOG.info(
+        "Updated table " + state.getTableName() + " state to " + state.getState() + " in META");
+  }
+
+  /**
+   * Construct PUT for given state
+   * @param state new state
+   */
+  public static Put makePutFromTableState(TableState state) {
+    long time = EnvironmentEdgeManager.currentTime();
+    Put put = new Put(state.getTableName().getName(), time);
+    put.add(getTableFamily(), getStateColumn(), state.convert().toByteArray());
+    return put;
+  }
+
+  /**
+   * Remove state for table from meta
+   * @param connection to use for deletion
+   * @param table to delete state for
+   */
+  public static void deleteTableState(Connection connection, TableName table)
+      throws IOException {
+    long time = EnvironmentEdgeManager.currentTime();
+    Delete delete = new Delete(table.getName());
+    delete.addColumns(getTableFamily(), getStateColumn(), time);
+    deleteFromMetaTable(connection, delete);
+    LOG.info("Deleted table " + table + " state from META");
+  }
+
+  /**
    * Performs an atomic multi-Mutate operation against the given table.
    */
   private static void multiMutate(Table table, byte[] row, Mutation... mutations)
@@ -1337,7 +1521,8 @@ public class MetaTableAccessor {
                                      HRegionInfo regionInfo, ServerName sn, long openSeqNum)
     throws IOException {
     // region replicas are kept in the primary region's row
-    Put put = new Put(getMetaKeyForRegion(regionInfo));
+    long time = EnvironmentEdgeManager.currentTime();
+    Put put = new Put(getMetaKeyForRegion(regionInfo), time);
     addLocation(put, sn, openSeqNum, regionInfo.getReplicaId());
     putToMetaTable(connection, put);
     LOG.info("Updated row " + regionInfo.getRegionNameAsString() +
@@ -1353,7 +1538,9 @@ public class MetaTableAccessor {
   public static void deleteRegion(Connection connection,
                                   HRegionInfo regionInfo)
     throws IOException {
+    long time = EnvironmentEdgeManager.currentTime();
     Delete delete = new Delete(regionInfo.getRegionName());
+    delete.addFamily(getCatalogFamily(), time);
     deleteFromMetaTable(connection, delete);
     LOG.info("Deleted " + regionInfo.getRegionNameAsString());
   }
@@ -1367,8 +1554,11 @@ public class MetaTableAccessor {
   public static void deleteRegions(Connection connection,
                                    List<HRegionInfo> regionsInfo) throws IOException {
     List<Delete> deletes = new ArrayList<Delete>(regionsInfo.size());
+    long time = EnvironmentEdgeManager.currentTime();
     for (HRegionInfo hri: regionsInfo) {
-      deletes.add(new Delete(hri.getRegionName()));
+      Delete e = new Delete(hri.getRegionName());
+      e.addFamily(getCatalogFamily(), time);
+      deletes.add(e);
     }
     deleteFromMetaTable(connection, deletes);
     LOG.info("Deleted " + regionsInfo);
@@ -1388,7 +1578,7 @@ public class MetaTableAccessor {
     List<Mutation> mutation = new ArrayList<Mutation>();
     if (regionsToRemove != null) {
       for (HRegionInfo hri: regionsToRemove) {
-        mutation.add(new Delete(hri.getRegionName()));
+        mutation.add(makeDeleteFromRegionInfo(hri));
       }
     }
     if (regionsToAdd != null) {
@@ -1431,9 +1621,10 @@ public class MetaTableAccessor {
    */
   public static void deleteMergeQualifiers(Connection connection,
                                            final HRegionInfo mergedRegion) throws IOException {
+    long time = EnvironmentEdgeManager.currentTime();
     Delete delete = new Delete(mergedRegion.getRegionName());
-    delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER);
-    delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER);
+    delete.addColumns(getCatalogFamily(), HConstants.MERGEA_QUALIFIER, time);
+    delete.addColumns(getCatalogFamily(), HConstants.MERGEB_QUALIFIER, time);
     deleteFromMetaTable(connection, delete);
     LOG.info("Deleted references in merged region "
       + mergedRegion.getRegionNameAsString() + ", qualifier="
@@ -1443,7 +1634,7 @@ public class MetaTableAccessor {
 
   private static Put addRegionInfo(final Put p, final HRegionInfo hri)
     throws IOException {
-    p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
+    p.addImmutable(getCatalogFamily(), HConstants.REGIONINFO_QUALIFIER,
       hri.toByteArray());
     return p;
   }
@@ -1452,20 +1643,20 @@ public class MetaTableAccessor {
     // using regionserver's local time as the timestamp of Put.
     // See: HBASE-11536
     long now = EnvironmentEdgeManager.currentTime();
-    p.addImmutable(HConstants.CATALOG_FAMILY, getServerColumn(replicaId), now,
+    p.addImmutable(getCatalogFamily(), getServerColumn(replicaId), now,
       Bytes.toBytes(sn.getHostAndPort()));
-    p.addImmutable(HConstants.CATALOG_FAMILY, getStartCodeColumn(replicaId), now,
+    p.addImmutable(getCatalogFamily(), getStartCodeColumn(replicaId), now,
       Bytes.toBytes(sn.getStartcode()));
-    p.addImmutable(HConstants.CATALOG_FAMILY, getSeqNumColumn(replicaId), now,
+    p.addImmutable(getCatalogFamily(), getSeqNumColumn(replicaId), now,
       Bytes.toBytes(openSeqNum));
     return p;
   }
 
   public static Put addEmptyLocation(final Put p, int replicaId) {
     long now = EnvironmentEdgeManager.currentTime();
-    p.addImmutable(HConstants.CATALOG_FAMILY, getServerColumn(replicaId), now, null);
-    p.addImmutable(HConstants.CATALOG_FAMILY, getStartCodeColumn(replicaId), now, null);
-    p.addImmutable(HConstants.CATALOG_FAMILY, getSeqNumColumn(replicaId), now, null);
+    p.addImmutable(getCatalogFamily(), getServerColumn(replicaId), now, null);
+    p.addImmutable(getCatalogFamily(), getStartCodeColumn(replicaId), now, null);
+    p.addImmutable(getCatalogFamily(), getSeqNumColumn(replicaId), now, null);
     return p;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java
new file mode 100644
index 0000000..3f44927
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A RetryingCallable for generic connection operations.
+ * @param <V> return type
+ */
+abstract class ConnectionCallable<V> implements RetryingCallable<V>, Closeable {
+  protected Connection connection;
+
+  public ConnectionCallable(final Connection connection) {
+    this.connection = connection;
+  }
+
+  @Override
+  public void prepare(boolean reload) throws IOException {
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+
+  @Override
+  public void throwable(Throwable t, boolean retrying) {
+  }
+
+  @Override
+  public String getExceptionMessageAdditionalDetail() {
+    return "";
+  }
+
+  @Override
+  public long sleep(long pause, int tries) {
+    return ConnectionUtils.getPauseTime(pause, tries);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
index dbd555c..e986156 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import javax.annotation.Nullable;
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -37,9 +38,12 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.BlockingRpcChannel;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -60,8 +64,6 @@ import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
-import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
-import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
 import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
 import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicyFactory;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
@@ -172,6 +174,7 @@ import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -179,11 +182,6 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.zookeeper.KeeperException;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.BlockingRpcChannel;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-
 /**
  * An internal, non-instantiable class that manages creation of {@link HConnection}s.
  */
@@ -929,30 +927,7 @@ final class ConnectionManager {
 
     @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() {
-        @Override
-        public boolean processRow(Result row) throws IOException {
-          HRegionInfo info = MetaScanner.getHRegionInfo(row);
-          if (info != null && !info.isSplitParent()) {
-            if (tableName.equals(info.getTable())) {
-              ServerName server = HRegionInfo.getServerName(row);
-              if (server == null) {
-                available.set(false);
-                return false;
-              }
-              regionCount.incrementAndGet();
-            } else if (tableName.compareTo(info.getTable()) < 0) {
-              // Return if we are done with the current table
-              return false;
-            }
-          }
-          return true;
-        }
-      };
-      MetaScanner.metaScan(this, visitor, tableName);
-      return available.get() && (regionCount.get() > 0);
+      return isTableAvailable(tableName, null);
     }
 
     @Override
@@ -961,44 +936,61 @@ final class ConnectionManager {
     }
 
     @Override
-    public boolean isTableAvailable(final TableName tableName, final byte[][] splitKeys)
+    public boolean isTableAvailable(final TableName tableName, @Nullable final byte[][] splitKeys)
         throws IOException {
-      final AtomicBoolean available = new AtomicBoolean(true);
-      final AtomicInteger regionCount = new AtomicInteger(0);
-      MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
-        @Override
-        public boolean processRow(Result row) throws IOException {
-          HRegionInfo info = MetaScanner.getHRegionInfo(row);
-          if (info != null && !info.isSplitParent()) {
-            if (tableName.equals(info.getTable())) {
-              ServerName server = HRegionInfo.getServerName(row);
-              if (server == null) {
-                available.set(false);
-                return false;
-              }
-              if (!Bytes.equals(info.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) {
-                for (byte[] splitKey : splitKeys) {
-                  // Just check if the splitkey is available
-                  if (Bytes.equals(info.getStartKey(), splitKey)) {
-                    regionCount.incrementAndGet();
-                    break;
-                  }
-                }
-              } else {
-                // Always empty start row should be counted
-                regionCount.incrementAndGet();
+      try {
+        if (!isTableEnabled(tableName)) {
+          LOG.debug("Table " + tableName + " not enabled");
+          return false;
+        }
+        ClusterConnection connection = getConnectionInternal(getConfiguration());
+        List<Pair<HRegionInfo, ServerName>> locations = MetaTableAccessor
+            .getTableRegionsAndLocations(connection, tableName, true);
+        int notDeployed = 0;
+        int regionCount = 0;
+        for (Pair<HRegionInfo, ServerName> pair : locations) {
+          HRegionInfo info = pair.getFirst();
+          if (pair.getSecond() == null) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Table " + tableName + " has not deployed region " + pair.getFirst()
+                  .getEncodedName());
+            }
+            notDeployed++;
+          } else if (splitKeys != null
+              && !Bytes.equals(info.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) {
+            for (byte[] splitKey : splitKeys) {
+              // Just check if the splitkey is available
+              if (Bytes.equals(info.getStartKey(), splitKey)) {
+                regionCount++;
+                break;
               }
-            } else if (tableName.compareTo(info.getTable()) < 0) {
-              // Return if we are done with the current table
-              return false;
             }
+          } else {
+            // Always empty start row should be counted
+            regionCount++;
+          }
+        }
+        if (notDeployed > 0) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Table " + tableName + " has " + notDeployed + " regions");
+          }
+          return false;
+        } else if (splitKeys != null && regionCount != splitKeys.length + 1) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Table " + tableName + " expected to have " + (splitKeys.length + 1)
+                + " regions, but only " + regionCount + " available");
+          }
+          return false;
+        } else {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Table " + tableName + " should be available");
           }
           return true;
         }
-      };
-      MetaScanner.metaScan(this, visitor, tableName);
-      // +1 needs to be added so that the empty start row is also taken into account
-      return available.get() && (regionCount.get() == splitKeys.length + 1);
+      } catch (TableNotFoundException tnfe) {
+        LOG.warn("Table " + tableName + " not enabled, it is not exists");
+        return false;
+      }
     }
 
     @Override
@@ -2485,7 +2477,7 @@ final class ConnectionManager {
       GetTableDescriptorsResponse htds;
       try {
         GetTableDescriptorsRequest req =
-          RequestConverter.buildGetTableDescriptorsRequest(tableName);
+            RequestConverter.buildGetTableDescriptorsRequest(tableName);
         htds = master.getTableDescriptors(null, req);
       } catch (ServiceException se) {
         throw ProtobufUtil.getRemoteException(se);
@@ -2510,16 +2502,11 @@ final class ConnectionManager {
 
     @Override
     public TableState getTableState(TableName tableName) throws IOException {
-      MasterKeepAliveConnection master = getKeepAliveMasterService();
-      try {
-        GetTableStateResponse resp = master.getTableState(null,
-                RequestConverter.buildGetTableStateRequest(tableName));
-        return TableState.convert(resp.getTableState());
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
-      } finally {
-        master.close();
-      }
+      ClusterConnection conn = getConnectionInternal(getConfiguration());
+      TableState tableState = MetaTableAccessor.getTableState(conn, tableName);
+      if (tableState == null)
+        throw new TableNotFoundException(tableName);
+      return tableState;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index d14e369..3acaaf9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.net.SocketTimeoutException;
@@ -286,7 +287,12 @@ public class HBaseAdmin implements Admin {
    */
   @Override
   public boolean tableExists(final TableName tableName) throws IOException {
-    return MetaTableAccessor.tableExists(connection, tableName);
+    return executeCallable(new ConnectionCallable<Boolean>(getConnection()) {
+      @Override
+      public Boolean call(int callTimeout) throws ServiceException, IOException {
+        return MetaTableAccessor.tableExists(connection, tableName);
+      }
+    });
   }
 
   public boolean tableExists(final byte[] tableName)
@@ -547,11 +553,11 @@ public class HBaseAdmin implements Admin {
     }
     int numRegs = (splitKeys == null ? 1 : splitKeys.length + 1) * desc.getRegionReplication();
     int prevRegCount = 0;
-    boolean doneWithMetaScan = false;
+    boolean tableWasEnabled = false;
     for (int tries = 0; tries < this.numRetries * this.retryLongerMultiplier;
       ++tries) {
-      if (!doneWithMetaScan) {
-        // Wait for new table to come on-line
+      if (tableWasEnabled) {
+        // Wait all table regions comes online
         final AtomicInteger actualRegCount = new AtomicInteger(0);
         MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
           @Override
@@ -599,17 +605,26 @@ public class HBaseAdmin implements Admin {
             tries = -1;
           }
         } else {
-          doneWithMetaScan = true;
-          tries = -1;
+          return;
         }
-      } else if (isTableEnabled(desc.getTableName())) {
-        return;
       } else {
-        try { // Sleep
-          Thread.sleep(getPauseTime(tries));
-        } catch (InterruptedException e) {
-          throw new InterruptedIOException("Interrupted when waiting" +
-            " for table to be enabled; meta scan was done");
+        try {
+          tableWasEnabled = isTableAvailable(desc.getTableName());
+        } catch (TableNotFoundException tnfe) {
+          LOG.debug(
+              "Table " + desc.getTableName() + " was not enabled, sleeping, still " + numRetries
+                  + " retries left");
+        }
+        if (tableWasEnabled) {
+          // no we will scan meta to ensure all regions are online
+          tries = -1;
+        } else {
+          try { // Sleep
+            Thread.sleep(getPauseTime(tries));
+          } catch (InterruptedException e) {
+            throw new InterruptedIOException("Interrupted when waiting" +
+                " for table to be enabled; meta scan was done");
+          }
         }
       }
     }
@@ -698,24 +713,11 @@ public class HBaseAdmin implements Admin {
     });
 
     int failures = 0;
-    // Wait until all regions deleted
     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
       try {
-        // Find whether all regions are deleted.
-        List<RegionLocations> regionLations =
-            MetaScanner.listTableRegionLocations(conf, connection, tableName);
-
-        // let us wait until hbase:meta table is updated and
-        // HMaster removes the table from its HTableDescriptors
-        if (regionLations == null || regionLations.size() == 0) {
-          HTableDescriptor htd = getTableDescriptorByTableName(tableName);
-
-          if (htd == null) {
-            // table could not be found in master - we are done.
-            tableExists = false;
-            break;
-          }
-        }
+        tableExists = tableExists(tableName);
+        if (!tableExists)
+          break;
       } catch (IOException ex) {
         failures++;
         if(failures >= numRetries - 1) {           // no more tries left
@@ -1109,9 +1111,17 @@ public class HBaseAdmin implements Admin {
    * @throws IOException if a remote or network exception occurs
    */
   @Override
-  public boolean isTableEnabled(TableName tableName) throws IOException {
+  public boolean isTableEnabled(final TableName tableName) throws IOException {
     checkTableExistence(tableName);
-    return connection.isTableEnabled(tableName);
+    return executeCallable(new ConnectionCallable<Boolean>(getConnection()) {
+      @Override
+      public Boolean call(int callTimeout) throws ServiceException, IOException {
+        TableState tableState = MetaTableAccessor.getTableState(connection, tableName);
+        if (tableState == null)
+          throw new TableNotFoundException(tableName);
+        return tableState.inStates(TableState.State.ENABLED);
+      }
+    });
   }
 
   public boolean isTableEnabled(byte[] tableName) throws IOException {
@@ -2296,10 +2306,15 @@ public class HBaseAdmin implements Admin {
    */
   private TableName checkTableExists(final TableName tableName)
       throws IOException {
-    if (!MetaTableAccessor.tableExists(connection, tableName)) {
-      throw new TableNotFoundException(tableName);
-    }
-    return tableName;
+    return executeCallable(new ConnectionCallable<TableName>(getConnection()) {
+      @Override
+      public TableName call(int callTimeout) throws ServiceException, IOException {
+        if (!MetaTableAccessor.tableExists(connection, tableName)) {
+          throw new TableNotFoundException(tableName);
+        }
+        return tableName;
+      }
+    });
   }
 
   /**
@@ -3667,7 +3682,8 @@ public class HBaseAdmin implements Admin {
     return QuotaRetriever.open(conf, filter);
   }
 
-  private <V> V executeCallable(MasterCallable<V> callable) throws IOException {
+  private <C extends RetryingCallable<V> & Closeable, V> V executeCallable(C callable)
+      throws IOException {
     RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller();
     try {
       return caller.callWithRetries(callable, operationTimeout);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
index be9b80c..77c90f5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableState.java
@@ -17,9 +17,11 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 
@@ -171,20 +173,30 @@ public class TableState {
   public HBaseProtos.TableState convert() {
     return HBaseProtos.TableState.newBuilder()
         .setState(this.state.convert())
-        .setTable(ProtobufUtil.toProtoTableName(this.tableName))
+        .setTable(ProtobufUtil.toProtoTableName(this.tableName)) // set for backward compatibility
         .setTimestamp(this.timestamp)
             .build();
   }
 
   /**
    * Covert from PB version of TableState
+   *
+   * @param tableName table this state of
    * @param tableState convert from
    * @return POJO
    */
-  public static TableState convert(HBaseProtos.TableState tableState) {
+  public static TableState convert(TableName tableName, HBaseProtos.TableState tableState) {
     TableState.State state = State.convert(tableState.getState());
-    return new TableState(ProtobufUtil.toTableName(tableState.getTable()),
-        state, tableState.getTimestamp());
+    return new TableState(tableName, state, tableState.getTimestamp());
+  }
+
+  public static TableState parseFrom(TableName tableName, byte[] bytes)
+      throws DeserializationException {
+    try {
+      return convert(tableName, HBaseProtos.TableState.parseFrom(bytes));
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
   }
 
   /**
@@ -200,4 +212,36 @@ public class TableState {
     }
     return false;
   }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    TableState that = (TableState) o;
+
+    if (timestamp != that.timestamp) return false;
+    if (state != that.state) return false;
+    if (tableName != null ? !tableName.equals(that.tableName) : that.tableName != null)
+      return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = (int) (timestamp ^ (timestamp >>> 32));
+    result = 31 * result + (tableName != null ? tableName.hashCode() : 0);
+    result = 31 * result + (state != null ? state.hashCode() : 0);
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return "TableState{" +
+        "timestamp=" + timestamp +
+        ", tableName=" + tableName +
+        ", state=" + state +
+        '}';
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index 2ee55f7..8a07397 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase;
 
 import static org.apache.hadoop.hbase.io.hfile.BlockType.MAGIC_LENGTH;
 
-import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
 import java.util.Arrays;
 import java.util.Collections;
@@ -451,6 +450,16 @@ public final class HConstants {
   /** The upper-half merge region column qualifier */
   public static final byte[] MERGEB_QUALIFIER = Bytes.toBytes("mergeB");
 
+  /** The catalog family as a string*/
+  public static final String TABLE_FAMILY_STR = "table";
+
+  /** The catalog family */
+  public static final byte [] TABLE_FAMILY = Bytes.toBytes(TABLE_FAMILY_STR);
+
+  /** The serialized table state qualifier */
+  public static final byte[] TABLE_STATE_QUALIFIER = Bytes.toBytes("state");
+
+
   /**
    * The meta table version column qualifier.
    * We keep current version of the meta table in this column in <code>-ROOT-</code>
@@ -738,7 +747,8 @@ public final class HConstants {
   /**
    * The client scanner timeout period in milliseconds.
    */
-  public static final String HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD = "hbase.client.scanner.timeout.period";
+  public static final String HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD =
+      "hbase.client.scanner.timeout.period";
 
   /**
    * Use {@link #HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD} instead.

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
index 2947f40..a96ef17 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
@@ -3450,15 +3450,15 @@ public final class HBaseProtos {
      */
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getSchemaOrBuilder();
 
-    // optional .TableState.State state = 2 [default = ENABLED];
+    // optional .TableState.State state = 2 [default = ENABLED, deprecated = true];
     /**
-     * <code>optional .TableState.State state = 2 [default = ENABLED];</code>
+     * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
      */
-    boolean hasState();
+    @java.lang.Deprecated boolean hasState();
     /**
-     * <code>optional .TableState.State state = 2 [default = ENABLED];</code>
+     * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState();
+    @java.lang.Deprecated org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState();
   }
   /**
    * Protobuf type {@code TableDescriptor}
@@ -3601,19 +3601,19 @@ public final class HBaseProtos {
       return schema_;
     }
 
-    // optional .TableState.State state = 2 [default = ENABLED];
+    // optional .TableState.State state = 2 [default = ENABLED, deprecated = true];
     public static final int STATE_FIELD_NUMBER = 2;
     private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State state_;
     /**
-     * <code>optional .TableState.State state = 2 [default = ENABLED];</code>
+     * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
      */
-    public boolean hasState() {
+    @java.lang.Deprecated public boolean hasState() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>optional .TableState.State state = 2 [default = ENABLED];</code>
+     * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState() {
+    @java.lang.Deprecated public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState() {
       return state_;
     }
 
@@ -4054,24 +4054,24 @@ public final class HBaseProtos {
         return schemaBuilder_;
       }
 
-      // optional .TableState.State state = 2 [default = ENABLED];
+      // optional .TableState.State state = 2 [default = ENABLED, deprecated = true];
       private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED;
       /**
-       * <code>optional .TableState.State state = 2 [default = ENABLED];</code>
+       * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
        */
-      public boolean hasState() {
+      @java.lang.Deprecated public boolean hasState() {
         return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>optional .TableState.State state = 2 [default = ENABLED];</code>
+       * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState() {
+      @java.lang.Deprecated public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State getState() {
         return state_;
       }
       /**
-       * <code>optional .TableState.State state = 2 [default = ENABLED];</code>
+       * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
        */
-      public Builder setState(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State value) {
+      @java.lang.Deprecated public Builder setState(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State value) {
         if (value == null) {
           throw new NullPointerException();
         }
@@ -4081,9 +4081,9 @@ public final class HBaseProtos {
         return this;
       }
       /**
-       * <code>optional .TableState.State state = 2 [default = ENABLED];</code>
+       * <code>optional .TableState.State state = 2 [default = ENABLED, deprecated = true];</code>
        */
-      public Builder clearState() {
+      @java.lang.Deprecated public Builder clearState() {
         bitField0_ = (bitField0_ & ~0x00000002);
         state_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableState.State.ENABLED;
         onChanged();
@@ -18197,52 +18197,52 @@ public final class HBaseProtos {
       "TableState.State\022\031\n\005table\030\002 \002(\0132\n.TableN" +
       "ame\022\021\n\ttimestamp\030\003 \001(\004\"?\n\005State\022\013\n\007ENABL" +
       "ED\020\000\022\014\n\010DISABLED\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENA",
-      "BLING\020\003\"Z\n\017TableDescriptor\022\034\n\006schema\030\001 \002" +
-      "(\0132\014.TableSchema\022)\n\005state\030\002 \001(\0162\021.TableS" +
-      "tate.State:\007ENABLED\"o\n\022ColumnFamilySchem" +
-      "a\022\014\n\004name\030\001 \002(\014\022#\n\nattributes\030\002 \003(\0132\017.By" +
-      "tesBytesPair\022&\n\rconfiguration\030\003 \003(\0132\017.Na" +
-      "meStringPair\"\232\001\n\nRegionInfo\022\021\n\tregion_id" +
-      "\030\001 \002(\004\022\036\n\ntable_name\030\002 \002(\0132\n.TableName\022\021" +
-      "\n\tstart_key\030\003 \001(\014\022\017\n\007end_key\030\004 \001(\014\022\017\n\007of" +
-      "fline\030\005 \001(\010\022\r\n\005split\030\006 \001(\010\022\025\n\nreplica_id" +
-      "\030\007 \001(\005:\0010\"1\n\014FavoredNodes\022!\n\014favored_nod",
-      "e\030\001 \003(\0132\013.ServerName\"\225\001\n\017RegionSpecifier" +
-      "\0222\n\004type\030\001 \002(\0162$.RegionSpecifier.RegionS" +
-      "pecifierType\022\r\n\005value\030\002 \002(\014\"?\n\023RegionSpe" +
-      "cifierType\022\017\n\013REGION_NAME\020\001\022\027\n\023ENCODED_R" +
-      "EGION_NAME\020\002\"%\n\tTimeRange\022\014\n\004from\030\001 \001(\004\022" +
-      "\n\n\002to\030\002 \001(\004\"A\n\nServerName\022\021\n\thost_name\030\001" +
-      " \002(\t\022\014\n\004port\030\002 \001(\r\022\022\n\nstart_code\030\003 \001(\004\"\033" +
-      "\n\013Coprocessor\022\014\n\004name\030\001 \002(\t\"-\n\016NameStrin" +
-      "gPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\",\n\rNa" +
-      "meBytesPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \001(\014",
-      "\"/\n\016BytesBytesPair\022\r\n\005first\030\001 \002(\014\022\016\n\006sec" +
-      "ond\030\002 \002(\014\",\n\rNameInt64Pair\022\014\n\004name\030\001 \001(\t" +
-      "\022\r\n\005value\030\002 \001(\003\"\314\001\n\023SnapshotDescription\022" +
-      "\014\n\004name\030\001 \002(\t\022\r\n\005table\030\002 \001(\t\022\030\n\rcreation" +
-      "_time\030\003 \001(\003:\0010\022.\n\004type\030\004 \001(\0162\031.SnapshotD" +
-      "escription.Type:\005FLUSH\022\017\n\007version\030\005 \001(\005\022" +
-      "\r\n\005owner\030\006 \001(\t\".\n\004Type\022\014\n\010DISABLED\020\000\022\t\n\005" +
-      "FLUSH\020\001\022\r\n\tSKIPFLUSH\020\002\"}\n\024ProcedureDescr" +
-      "iption\022\021\n\tsignature\030\001 \002(\t\022\020\n\010instance\030\002 " +
-      "\001(\t\022\030\n\rcreation_time\030\003 \001(\003:\0010\022&\n\rconfigu",
-      "ration\030\004 \003(\0132\017.NameStringPair\"\n\n\010EmptyMs" +
-      "g\"\033\n\007LongMsg\022\020\n\010long_msg\030\001 \002(\003\"\037\n\tDouble" +
-      "Msg\022\022\n\ndouble_msg\030\001 \002(\001\"\'\n\rBigDecimalMsg" +
-      "\022\026\n\016bigdecimal_msg\030\001 \002(\014\"5\n\004UUID\022\026\n\016leas" +
-      "t_sig_bits\030\001 \002(\004\022\025\n\rmost_sig_bits\030\002 \002(\004\"" +
-      "K\n\023NamespaceDescriptor\022\014\n\004name\030\001 \002(\014\022&\n\r" +
-      "configuration\030\002 \003(\0132\017.NameStringPair\"$\n\020" +
-      "RegionServerInfo\022\020\n\010infoPort\030\001 \001(\005*r\n\013Co" +
-      "mpareType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t" +
-      "\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_E",
-      "QUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006*n\n\010TimeUn" +
-      "it\022\017\n\013NANOSECONDS\020\001\022\020\n\014MICROSECONDS\020\002\022\020\n" +
-      "\014MILLISECONDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007MINUTES\020" +
-      "\005\022\t\n\005HOURS\020\006\022\010\n\004DAYS\020\007B>\n*org.apache.had" +
-      "oop.hbase.protobuf.generatedB\013HBaseProto" +
-      "sH\001\240\001\001"
+      "BLING\020\003\"^\n\017TableDescriptor\022\034\n\006schema\030\001 \002" +
+      "(\0132\014.TableSchema\022-\n\005state\030\002 \001(\0162\021.TableS" +
+      "tate.State:\007ENABLEDB\002\030\001\"o\n\022ColumnFamilyS" +
+      "chema\022\014\n\004name\030\001 \002(\014\022#\n\nattributes\030\002 \003(\0132" +
+      "\017.BytesBytesPair\022&\n\rconfiguration\030\003 \003(\0132" +
+      "\017.NameStringPair\"\232\001\n\nRegionInfo\022\021\n\tregio" +
+      "n_id\030\001 \002(\004\022\036\n\ntable_name\030\002 \002(\0132\n.TableNa" +
+      "me\022\021\n\tstart_key\030\003 \001(\014\022\017\n\007end_key\030\004 \001(\014\022\017" +
+      "\n\007offline\030\005 \001(\010\022\r\n\005split\030\006 \001(\010\022\025\n\nreplic" +
+      "a_id\030\007 \001(\005:\0010\"1\n\014FavoredNodes\022!\n\014favored",
+      "_node\030\001 \003(\0132\013.ServerName\"\225\001\n\017RegionSpeci" +
+      "fier\0222\n\004type\030\001 \002(\0162$.RegionSpecifier.Reg" +
+      "ionSpecifierType\022\r\n\005value\030\002 \002(\014\"?\n\023Regio" +
+      "nSpecifierType\022\017\n\013REGION_NAME\020\001\022\027\n\023ENCOD" +
+      "ED_REGION_NAME\020\002\"%\n\tTimeRange\022\014\n\004from\030\001 " +
+      "\001(\004\022\n\n\002to\030\002 \001(\004\"A\n\nServerName\022\021\n\thost_na" +
+      "me\030\001 \002(\t\022\014\n\004port\030\002 \001(\r\022\022\n\nstart_code\030\003 \001" +
+      "(\004\"\033\n\013Coprocessor\022\014\n\004name\030\001 \002(\t\"-\n\016NameS" +
+      "tringPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\"," +
+      "\n\rNameBytesPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002",
+      " \001(\014\"/\n\016BytesBytesPair\022\r\n\005first\030\001 \002(\014\022\016\n" +
+      "\006second\030\002 \002(\014\",\n\rNameInt64Pair\022\014\n\004name\030\001" +
+      " \001(\t\022\r\n\005value\030\002 \001(\003\"\314\001\n\023SnapshotDescript" +
+      "ion\022\014\n\004name\030\001 \002(\t\022\r\n\005table\030\002 \001(\t\022\030\n\rcrea" +
+      "tion_time\030\003 \001(\003:\0010\022.\n\004type\030\004 \001(\0162\031.Snaps" +
+      "hotDescription.Type:\005FLUSH\022\017\n\007version\030\005 " +
+      "\001(\005\022\r\n\005owner\030\006 \001(\t\".\n\004Type\022\014\n\010DISABLED\020\000" +
+      "\022\t\n\005FLUSH\020\001\022\r\n\tSKIPFLUSH\020\002\"}\n\024ProcedureD" +
+      "escription\022\021\n\tsignature\030\001 \002(\t\022\020\n\010instanc" +
+      "e\030\002 \001(\t\022\030\n\rcreation_time\030\003 \001(\003:\0010\022&\n\rcon",
+      "figuration\030\004 \003(\0132\017.NameStringPair\"\n\n\010Emp" +
+      "tyMsg\"\033\n\007LongMsg\022\020\n\010long_msg\030\001 \002(\003\"\037\n\tDo" +
+      "ubleMsg\022\022\n\ndouble_msg\030\001 \002(\001\"\'\n\rBigDecima" +
+      "lMsg\022\026\n\016bigdecimal_msg\030\001 \002(\014\"5\n\004UUID\022\026\n\016" +
+      "least_sig_bits\030\001 \002(\004\022\025\n\rmost_sig_bits\030\002 " +
+      "\002(\004\"K\n\023NamespaceDescriptor\022\014\n\004name\030\001 \002(\014" +
+      "\022&\n\rconfiguration\030\002 \003(\0132\017.NameStringPair" +
+      "\"$\n\020RegionServerInfo\022\020\n\010infoPort\030\001 \001(\005*r" +
+      "\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL" +
+      "\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_",
+      "OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006*n\n\010Ti" +
+      "meUnit\022\017\n\013NANOSECONDS\020\001\022\020\n\014MICROSECONDS\020" +
+      "\002\022\020\n\014MILLISECONDS\020\003\022\013\n\007SECONDS\020\004\022\013\n\007MINU" +
+      "TES\020\005\022\t\n\005HOURS\020\006\022\010\n\004DAYS\020\007B>\n*org.apache" +
+      ".hadoop.hbase.protobuf.generatedB\013HBaseP" +
+      "rotosH\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-protocol/src/main/protobuf/HBase.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/HBase.proto b/hbase-protocol/src/main/protobuf/HBase.proto
index c3c8c6a..1566846 100644
--- a/hbase-protocol/src/main/protobuf/HBase.proto
+++ b/hbase-protocol/src/main/protobuf/HBase.proto
@@ -55,14 +55,14 @@ message TableState {
   }
   // This is the table's state.
   required State state = 1;
-  required TableName table = 2;
+  required TableName table = 2 [deprecated = true];
   optional uint64 timestamp = 3;
 }
 
 /** On HDFS representation of table state. */
 message TableDescriptor {
   required TableSchema schema = 1;
-  optional TableState.State state = 2 [ default = ENABLED ];
+  optional TableState.State state = 2 [ default = ENABLED, deprecated = true ];
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptor.java
index d27bfb7..d1935db 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptor.java
@@ -17,12 +17,13 @@
  */
 package org.apache.hadoop.hbase;
 
+import javax.annotation.Nullable;
 import java.io.IOException;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.InvalidProtocolBufferException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -35,15 +36,23 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
 @InterfaceAudience.Private
 public class TableDescriptor {
   private HTableDescriptor hTableDescriptor;
+  /**
+   * Don't use, state was moved to meta, use MetaTableAccessor instead
+   * @deprecated state was moved to meta
+   */
+  @Deprecated
+  @Nullable
   private TableState.State tableState;
 
   /**
    * Creates TableDescriptor with all fields.
    * @param hTableDescriptor HTableDescriptor to use
    * @param tableState table state
+   * @deprecated state was moved to meta
    */
+  @Deprecated
   public TableDescriptor(HTableDescriptor hTableDescriptor,
-      TableState.State tableState) {
+      @Nullable TableState.State tableState) {
     this.hTableDescriptor = hTableDescriptor;
     this.tableState = tableState;
   }
@@ -69,22 +78,35 @@ public class TableDescriptor {
     this.hTableDescriptor = hTableDescriptor;
   }
 
+  /**
+   * @return table state
+   * @deprecated state was moved to meta
+   */
+  @Deprecated
+  @Nullable
   public TableState.State getTableState() {
     return tableState;
   }
 
-  public void setTableState(TableState.State tableState) {
+  /**
+   * @param tableState state to set for table
+   * @deprecated state was moved to meta
+   */
+  @Deprecated
+  public void setTableState(@Nullable TableState.State tableState) {
     this.tableState = tableState;
   }
 
   /**
    * Convert to PB.
    */
+  @SuppressWarnings("deprecation")
   public HBaseProtos.TableDescriptor convert() {
-    return HBaseProtos.TableDescriptor.newBuilder()
-        .setSchema(hTableDescriptor.convert())
-        .setState(tableState.convert())
-        .build();
+    HBaseProtos.TableDescriptor.Builder builder = HBaseProtos.TableDescriptor.newBuilder()
+        .setSchema(hTableDescriptor.convert());
+    if (tableState!= null)
+      builder.setState(tableState.convert());
+    return builder.build();
   }
 
   /**
@@ -92,7 +114,9 @@ public class TableDescriptor {
    */
   public static TableDescriptor convert(HBaseProtos.TableDescriptor proto) {
     HTableDescriptor hTableDescriptor = HTableDescriptor.convert(proto.getSchema());
-    TableState.State state = TableState.State.convert(proto.getState());
+    TableState.State state = proto.hasState()?
+        TableState.State.convert(proto.getState())
+        :null;
     return new TableDescriptor(hTableDescriptor, state);
   }
 
@@ -170,6 +194,17 @@ public class TableDescriptor {
                 .setBloomFilterType(BloomType.NONE)
                     // Enable cache of data blocks in L1 if more than one caching tier deployed:
                     // e.g. if using CombinedBlockCache (BucketCache).
+                .setCacheDataInL1(true),
+            new HColumnDescriptor(HConstants.TABLE_FAMILY)
+                // Ten is arbitrary number.  Keep versions to help debugging.
+                .setMaxVersions(10)
+                .setInMemory(true)
+                .setBlocksize(8 * 1024)
+                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+                    // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+                .setBloomFilterType(BloomType.NONE)
+                    // Enable cache of data blocks in L1 if more than one caching tier deployed:
+                    // e.g. if using CombinedBlockCache (BucketCache).
                 .setCacheDataInL1(true)
         }) {
     };

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index fac1ac9..064771c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -198,6 +198,7 @@ public class RpcServer implements RpcServerInterface {
 
   protected final InetSocketAddress bindAddress;
   protected int port;                             // port we listen on
+  protected InetSocketAddress address;            // inet address we listen on
   private int readThreads;                        // number of read threads
   protected int maxIdleTime;                      // the maximum idle time after
                                                   // which a client may be
@@ -528,6 +529,7 @@ public class RpcServer implements RpcServerInterface {
       // Bind the server socket to the binding addrees (can be different from the default interface)
       bind(acceptChannel.socket(), bindAddress, backlogLength);
       port = acceptChannel.socket().getLocalPort(); //Could be an ephemeral port
+      address = (InetSocketAddress)acceptChannel.socket().getLocalSocketAddress();
       // create a selector;
       selector= Selector.open();
 
@@ -754,7 +756,7 @@ public class RpcServer implements RpcServerInterface {
     }
 
     InetSocketAddress getAddress() {
-      return (InetSocketAddress)acceptChannel.socket().getLocalSocketAddress();
+      return address;
     }
 
     void doAccept(SelectionKey key) throws IOException, OutOfMemoryError {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
index 4d9ff13..f861529 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.RegionLocations;
-import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
@@ -99,7 +98,7 @@ import com.google.common.annotations.VisibleForTesting;
 public class AssignmentManager {
   private static final Log LOG = LogFactory.getLog(AssignmentManager.class);
 
-  protected final Server server;
+  protected final MasterServices server;
 
   private ServerManager serverManager;
 
@@ -130,8 +129,8 @@ public class AssignmentManager {
   private final int maximumAttempts;
 
   /**
-   * The sleep time for which the assignment will wait before retrying in case of hbase:meta assignment
-   * failure due to lack of availability of region plan or bad region plan
+   * The sleep time for which the assignment will wait before retrying in case of
+   * hbase:meta assignment failure due to lack of availability of region plan or bad region plan
    */
   private final long sleepTimeBeforeRetryingMetaAssignment;
 
@@ -209,7 +208,7 @@ public class AssignmentManager {
    * @param tableLockManager TableLock manager
    * @throws IOException
    */
-  public AssignmentManager(Server server, ServerManager serverManager,
+  public AssignmentManager(MasterServices server, ServerManager serverManager,
       final LoadBalancer balancer,
       final ExecutorService service, MetricsMaster metricsMaster,
       final TableLockManager tableLockManager,
@@ -1570,7 +1569,7 @@ public class AssignmentManager {
             TableState.State.ENABLING);
 
     // Region assignment from META
-    List<Result> results = MetaTableAccessor.fullScanOfMeta(server.getConnection());
+    List<Result> results = MetaTableAccessor.fullScanRegions(server.getConnection());
     // Get any new but slow to checkin region server that joined the cluster
     Set<ServerName> onlineServers = serverManager.getOnlineServers().keySet();
     // Set of offline servers to be returned

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 020d6fb..61a1c66 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -430,6 +430,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     return connector.getLocalPort();
   }
 
+  @Override
+  protected TableDescriptors getFsTableDescriptors() throws IOException {
+    return super.getFsTableDescriptors();
+  }
+
   /**
    * For compatibility, if failed with regionserver credentials, try the master one
    */
@@ -629,9 +634,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
 
     // Invalidate all write locks held previously
     this.tableLockManager.reapWriteLocks();
-
     this.tableStateManager = new TableStateManager(this);
-    this.tableStateManager.start();
 
     status.setStatus("Initializing ZK system trackers");
     initializeZKBasedSystemTrackers();
@@ -869,7 +872,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       assigned++;
     }
 
-    if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) enableMeta(TableName.META_TABLE_NAME);
+    if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID)
+      getTableStateManager().setTableState(TableName.META_TABLE_NAME, TableState.State.ENABLED);
+    // TODO: should we prevent from using state manager before meta was initialized?
+    // tableStateManager.start();
 
     if ((RecoveryMode.LOG_REPLAY == this.getMasterFileSystem().getLogRecoveryMode())
         && (!previouslyFailedMetaRSs.isEmpty())) {
@@ -878,6 +884,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       this.fileSystemManager.splitMetaLog(previouslyFailedMetaRSs);
     }
 
+    this.assignmentManager.setEnabledTable(TableName.META_TABLE_NAME);
+    tableStateManager.start();
+
     // Make sure a hbase:meta location is set. We need to enable SSH here since
     // if the meta region server is died at this time, we need it to be re-assigned
     // by SSH so that system tables can be assigned.
@@ -934,13 +943,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     }
   }
 
-  private void enableMeta(TableName metaTableName) {
-    if (!this.tableStateManager.isTableState(metaTableName,
-            TableState.State.ENABLED)) {
-      this.assignmentManager.setEnabledTable(metaTableName);
-    }
-  }
-
   /**
    * This function returns a set of region server names under hbase:meta recovering region ZK node
    * @return Set of meta server names which were recorded in ZK
@@ -1173,7 +1175,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
           if (rpCount < plans.size() &&
               // if performing next balance exceeds cutoff time, exit the loop
               (System.currentTimeMillis() + (totalRegPlanExecTime / rpCount)) > cutoffTime) {
-            //TODO: After balance, there should not be a cutoff time (keeping it as a security net for now)
+            //TODO: After balance, there should not be a cutoff time (keeping it as
+            // a security net for now)
             LOG.debug("No more balancing till next balance run; maximumBalanceTime=" +
               maximumBalanceTime);
             break;
@@ -1463,7 +1466,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
           LOG.fatal("Failed to become active master", t);
           // HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
           if (t instanceof NoClassDefFoundError &&
-              t.getMessage().contains("org/apache/hadoop/hdfs/protocol/FSConstants$SafeModeAction")) {
+              t.getMessage()
+                  .contains("org/apache/hadoop/hdfs/protocol/FSConstants$SafeModeAction")) {
             // improved error message for this special case
             abort("HBase is having a problem with its Hadoop jars.  You may need to "
               + "recompile HBase against Hadoop version "
@@ -2192,15 +2196,18 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
         }
 
         for (HTableDescriptor desc: htds) {
-          if (includeSysTables || !desc.getTableName().isSystemTable()) {
+          if (tableStateManager.isTablePresent(desc.getTableName())
+              && (includeSysTables || !desc.getTableName().isSystemTable())) {
             descriptors.add(desc);
           }
         }
       } else {
         for (TableName s: tableNameList) {
-          HTableDescriptor desc = tableDescriptors.get(s);
-          if (desc != null) {
-            descriptors.add(desc);
+          if (tableStateManager.isTablePresent(s)) {
+            HTableDescriptor desc = tableDescriptors.get(s);
+            if (desc != null) {
+              descriptors.add(desc);
+            }
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
index f979403..c4eecfa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.master;
 
+import javax.annotation.Nullable;
 import java.util.List;
 import java.util.Map;
 
@@ -88,6 +89,7 @@ public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObse
    * @param servers
    * @return List of plans
    */
+  @Nullable
   Map<ServerName, List<HRegionInfo>> retainAssignment(
     Map<HRegionInfo, ServerName> regions,
     List<ServerName> servers

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
index 4d72312..78e4c11 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
@@ -471,7 +471,7 @@ public class MasterFileSystem {
     // we should get them from registry.
     FSTableDescriptors fsd = new FSTableDescriptors(c, fs, rd);
     fsd.createTableDescriptor(
-        new TableDescriptor(fsd.get(TableName.META_TABLE_NAME), TableState.State.ENABLING));
+        new TableDescriptor(fsd.get(TableName.META_TABLE_NAME)));
 
     return rd;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 0e81461..4af53a4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -850,8 +850,6 @@ public class MasterRpcServices extends RSRpcServices
       TableName tableName = ProtobufUtil.toTableName(request.getTableName());
       TableState.State state = master.getTableStateManager()
               .getTableState(tableName);
-      if (state == null)
-        throw new TableNotFoundException(tableName);
       MasterProtos.GetTableStateResponse.Builder builder =
               MasterProtos.GetTableStateResponse.newBuilder();
       builder.setTableState(new TableState(tableName, state).convert());

http://git-wip-us.apache.org/repos/asf/hbase/blob/fa852c4c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
index 9dd412c..df61b45 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
@@ -230,7 +230,8 @@ public class RegionStateStore {
         }
       }
       // Called when meta is not on master
-      multiHConnection.processBatchCallback(Arrays.asList(put), TableName.META_TABLE_NAME, null, null);
+      multiHConnection.processBatchCallback(Arrays.asList(put),
+          TableName.META_TABLE_NAME, null, null);
 
     } catch (IOException ioe) {
       LOG.error("Failed to persist region state " + newState, ioe);