Repository: hbase
Updated Branches:
  refs/heads/HBASE-19064 e7d56a3ef -> 33d0606a2 (forced update)


HBASE-16060 1.x clients cannot access table state talking to 2.0 cluster

This patch adds mirroring of table state out to zookeeper. HBase-1.x
clients look for table state in zookeeper, not in hbase:meta where
hbase-2.x maintains table state.

The patch also moves and refactors the 'migration' code that was put in
place by HBASE-13032.

D 
hbase-client/src/main/java/org/apache/hadoop/hbase/CoordinatedStateException.java
 Unused.

M hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
 Move table state migration code from Master startup out to
TableStateManager where it belongs. Also start
MirroringTableStateManager dependent on config.

A 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MirroringTableStateManager.java

M 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
 Move migration from zookeeper of table state in here. Also plumb in
mechanism so subclass can get a chance to look at table state as we do
the startup fixup full-table scan of meta.

M 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
 Bug-fix. Now we create regions in CLOSED state but we fail to check
table state; were presuming table always enabled. Meant on startup
there'd be an unassigned region that never got assigned.

A 
hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMirroringTableStateManager.java
 Test migration and mirroring.


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

Branch: refs/heads/HBASE-19064
Commit: 67b69fb2c70d3a56ac45f59d57b7f2778094a566
Parents: 8ff783f
Author: Michael Stack <st...@apache.org>
Authored: Thu Feb 8 00:19:06 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Mon Feb 12 08:47:02 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/CoordinatedStateException.java |  46 -------
 .../hadoop/hbase/zookeeper/ZNodePaths.java      |   3 +
 .../src/main/protobuf/ZooKeeper.proto           |   2 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  27 ++--
 .../hbase/master/MasterMetaBootstrap.java       |   3 +
 .../master/MirroringTableStateManager.java      | 109 +++++++++++++++
 .../hadoop/hbase/master/TableStateManager.java  | 132 ++++++++++++++++---
 .../master/assignment/AssignmentManager.java    |  11 +-
 .../master/procedure/DeleteTableProcedure.java  |   5 +-
 .../master/procedure/ProcedureSyncWait.java     |   3 +-
 .../procedure/TruncateTableProcedure.java       |   5 +-
 .../hadoop/hbase/util/ZKDataMigrator.java       |  11 +-
 .../hbase/master/TestMasterNoCluster.java       |   3 +-
 .../master/TestMirroringTableStateManager.java  | 105 +++++++++++++++
 .../hbase/master/TestTableStateManager.java     |  70 +++++-----
 15 files changed, 405 insertions(+), 130 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/67b69fb2/hbase-client/src/main/java/org/apache/hadoop/hbase/CoordinatedStateException.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/CoordinatedStateException.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/CoordinatedStateException.java
deleted file mode 100644
index fc0c4bc..0000000
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/CoordinatedStateException.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- *
- * 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;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.exceptions.HBaseException;
-
-/**
- * Thrown by operations requiring coordination state access or manipulation
- * when internal error within coordination engine (or other internal 
implementation) occurs.
- */
-@InterfaceAudience.Private
-@SuppressWarnings("serial")
-public class CoordinatedStateException extends HBaseException {
-  public CoordinatedStateException() {
-    super();
-  }
-
-  public CoordinatedStateException(final String message) {
-    super(message);
-  }
-
-  public CoordinatedStateException(final String message, final Throwable t) {
-    super(message, t);
-  }
-
-  public CoordinatedStateException(final Throwable t) {
-    super(t);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/67b69fb2/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
index 9e7e51a..32792f6 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZNodePaths.java
@@ -59,6 +59,9 @@ public class ZNodePaths {
   // znode containing the current cluster state
   public final String clusterStateZNode;
   // znode used for table disabling/enabling
+  // Still used in hbase2 by MirroringTableStateManager; it mirrors internal 
table state out to
+  // zookeeper for hbase1 clients to make use of. If no hbase1 clients 
disable. See
+  // MirroringTableStateManager. To be removed in hbase3.
   @Deprecated
   public final String tableZNode;
   // znode containing the unique cluster ID

http://git-wip-us.apache.org/repos/asf/hbase/blob/67b69fb2/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto 
b/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto
index 2a87ef6..383388b 100644
--- a/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto
@@ -85,7 +85,7 @@ message SplitLogTask {
 
 /**
  * The znode that holds state of table.
- * Deprected, table state is stored in table descriptor on HDFS.
+ * Deprected, table state is stored in hbase:meta since 2.0.0.
  */
 message DeprecatedTableState {
   // Table's current state

http://git-wip-us.apache.org/repos/asf/hbase/blob/67b69fb2/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 ecf6d63..02fbc02 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
@@ -61,7 +61,6 @@ import org.apache.hadoop.hbase.ClusterId;
 import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterMetricsBuilder;
-import org.apache.hadoop.hbase.CoordinatedStateException;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
@@ -186,7 +185,6 @@ import org.apache.hadoop.hbase.util.ModifyRegionUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.VersionInfo;
-import org.apache.hadoop.hbase.util.ZKDataMigrator;
 import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 import org.apache.hadoop.hbase.zookeeper.MasterMaintenanceModeTracker;
@@ -720,7 +718,7 @@ public class HMaster extends HRegionServer implements 
MasterServices {
    * Initialize all ZK based system trackers.
    */
   void initializeZKBasedSystemTrackers() throws IOException, 
InterruptedException, KeeperException,
-      CoordinatedStateException, ReplicationException {
+      ReplicationException {
     this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
     this.normalizer = RegionNormalizerFactory.getRegionNormalizer(conf);
     this.normalizer.setMasterServices(this);
@@ -784,7 +782,7 @@ public class HMaster extends HRegionServer implements 
MasterServices {
    * </ol>
    */
   private void finishActiveMasterInitialization(MonitoredTask status) throws 
IOException,
-          InterruptedException, KeeperException, CoordinatedStateException, 
ReplicationException {
+          InterruptedException, KeeperException, ReplicationException {
     Thread zombieDetector = new Thread(new InitializationMonitor(this),
         "ActiveMasterInitializationMonitor-" + System.currentTimeMillis());
     zombieDetector.setDaemon(true);
@@ -823,7 +821,12 @@ public class HMaster extends HRegionServer implements 
MasterServices {
 
     // This manager is started AFTER hbase:meta is confirmed on line.
     // See inside metaBootstrap.recoverMeta(); below. Shouldn't be so cryptic!
-    this.tableStateManager = new TableStateManager(this);
+    // hbase.mirror.table.state.to.zookeeper is so hbase1 clients can connect. 
They read table
+    // state from zookeeper while hbase2 reads it from hbase:meta. Disable if 
no hbase1 clients.
+    this.tableStateManager =
+        
this.conf.getBoolean(MirroringTableStateManager.MIRROR_TABLE_STATE_TO_ZK_KEY, 
true)?
+        new MirroringTableStateManager(this):
+        new TableStateManager(this);
 
     status.setStatus("Initializing ZK system trackers");
     initializeZKBasedSystemTrackers();
@@ -878,12 +881,11 @@ public class HMaster extends HRegionServer implements 
MasterServices {
       return;
     }
 
-    // we recover hbase:meta region servers inside master initialization and
-    // handle other failed servers in SSH in order to start up master node ASAP
+    // Bring up hbase:meta. recoverMeta is a blocking call waiting until 
hbase:meta is deployed.
+    // It also starts the TableStateManager.
     MasterMetaBootstrap metaBootstrap = createMetaBootstrap(this, status);
     metaBootstrap.recoverMeta();
 
-
     //Initialize after meta as it scans meta
     if (favoredNodesManager != null) {
       SnapshotOfRegionAssignmentFromMeta snapshotOfRegionAssignment =
@@ -892,15 +894,6 @@ public class HMaster extends HRegionServer implements 
MasterServices {
       favoredNodesManager.initialize(snapshotOfRegionAssignment);
     }
 
-    // migrating existent table state from zk, so splitters
-    // and recovery process treat states properly.
-    for (Map.Entry<TableName, TableState.State> entry : ZKDataMigrator
-        .queryForTableStates(getZooKeeper()).entrySet()) {
-      LOG.info("Converting state from zk to new states:" + entry);
-      tableStateManager.setTableState(entry.getKey(), entry.getValue());
-    }
-    ZKUtil.deleteChildrenRecursively(getZooKeeper(), 
getZooKeeper().znodePaths.tableZNode);
-
     status.setStatus("Submitting log splitting work for previously failed 
region servers");
     metaBootstrap.processDeadServers();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/67b69fb2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
index b8972c7..eab937d 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
@@ -53,8 +53,11 @@ public class MasterMetaBootstrap {
   }
 
   public void recoverMeta() throws InterruptedException, IOException {
+    // This is a blocking call that waits until hbase:meta is deployed.
     master.recoverMeta();
+    // Now we can start the TableStateManager. It is backed by hbase:meta.
     master.getTableStateManager().start();
+    // Enable server crash procedure handling
     enableCrashedServerProcessing(false);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/67b69fb2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MirroringTableStateManager.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MirroringTableStateManager.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MirroringTableStateManager.java
new file mode 100644
index 0000000..752c941
--- /dev/null
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MirroringTableStateManager.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A subclass of TableStateManager that mirrors change in state out to 
zookeeper for hbase-1.x
+ * clients to pick up; hbase-1.x clients read table state of zookeeper rather 
than from hbase:meta
+ * as hbase-2.x clients do. Set "hbase.mirror.table.state.to.zookeeper" to 
false to disable
+ * mirroring. See in HMaster where we make the choice. The below does zk 
updates on a best-effort
+ * basis only. If we fail updating zk we keep going because only hbase1 
clients suffer; we'll just
+ * log at WARN level.
+ * @deprecated Since 2.0.0. To be removed in 3.0.0.
+ */
+@Deprecated
+@InterfaceAudience.Private
+public class MirroringTableStateManager extends TableStateManager {
+  private static final Logger LOG = 
LoggerFactory.getLogger(MirroringTableStateManager.class);
+
+  /**
+   * Set this key to true in Configuration to enable mirroring of table state 
out to zookeeper
+   * so hbase-1.x clients can pick-up table state.
+   */
+  static final String MIRROR_TABLE_STATE_TO_ZK_KEY = 
"hbase.mirror.table.state.to.zookeeper";
+
+  public MirroringTableStateManager(MasterServices master) {
+    super(master);
+  }
+
+  protected void updateMetaState(TableName tableName, TableState.State 
newState)
+      throws IOException {
+    // Take the lock. Its reentrant. Calls to super will take same lock.
+    lock.writeLock().lock();
+    try {
+      super.updateMetaState(tableName, newState);
+      updateZooKeeper(new TableState(tableName, newState));
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
+  public void setDeletedTable(TableName tableName) throws IOException {
+    lock.writeLock().lock();
+    try {
+      super.setDeletedTable(tableName);
+      deleteZooKeeper(tableName);
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
+  private void updateZooKeeper(TableState tableState) throws IOException {
+    if (tableState == null || tableState.getState() == null) {
+      return;
+    }
+    String znode = 
ZNodePaths.joinZNode(this.master.getZooKeeper().getZNodePaths().tableZNode,
+        tableState.getTableName().getNameAsString());
+    try {
+      // Make sure znode exists.
+      if (ZKUtil.checkExists(this.master.getZooKeeper(), znode) == -1) {
+        ZKUtil.createAndFailSilent(this.master.getZooKeeper(), znode);
+      }
+      // Now set newState
+      ZooKeeperProtos.DeprecatedTableState.Builder builder =
+          ZooKeeperProtos.DeprecatedTableState.newBuilder();
+      builder.setState(ZooKeeperProtos.DeprecatedTableState.State.
+          valueOf(tableState.getState().toString()));
+      byte [] data = 
ProtobufUtil.prependPBMagic(builder.build().toByteArray());
+      ZKUtil.setData(this.master.getZooKeeper(), znode, data);
+    } catch (KeeperException e) {
+      // Only hbase1 clients suffer if this fails.
+      LOG.warn("Failed setting table state to zookeeper mirrored for hbase-1.x 
clients", e);
+    }
+  }
+
+  // This method is called by the super class on each row it finds in the 
hbase:meta table with
+  // table state in it.
+  @Override
+  protected void fixTableState(TableState tableState) throws IOException {
+    updateZooKeeper(tableState);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/67b69fb2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
index 021a1a3..0d89eef 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -26,6 +26,9 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
+import org.apache.hadoop.hbase.util.ZKDataMigrator;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import edu.umd.cs.findbugs.annotations.NonNull;
 import edu.umd.cs.findbugs.annotations.Nullable;
@@ -35,6 +38,7 @@ import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.client.Connection;
@@ -43,15 +47,22 @@ import org.apache.hadoop.hbase.client.TableState;
 
 /**
  * This is a helper class used to manage table states.
- * States persisted in tableinfo and cached internally.
+ * This class uses hbase:meta as its store for table state so hbase:meta must 
be online before
+ * {@link #start()} is called.
  * TODO: Cache state. Cut down on meta looksups.
  */
+// TODO: Make this a guava Service
 @InterfaceAudience.Private
 public class TableStateManager {
   private static final Logger LOG = 
LoggerFactory.getLogger(TableStateManager.class);
+  /**
+   * Set this key to false in Configuration to disable migrating table state 
from zookeeper
+   * so hbase:meta table.
+   */
+  static final String MIGRATE_TABLE_STATE_FROM_ZK_KEY = 
"hbase.migrate.table.state.from.zookeeper";
 
-  private final ReadWriteLock lock = new ReentrantReadWriteLock();
-  private final MasterServices master;
+  final ReadWriteLock lock = new ReentrantReadWriteLock();
+  final MasterServices master;
 
   public TableStateManager(MasterServices master) {
     this.master = master;
@@ -71,7 +82,6 @@ public class TableStateManager {
     } finally {
       lock.writeLock().unlock();
     }
-
   }
 
   /**
@@ -140,8 +150,9 @@ public class TableStateManager {
   }
 
   public void setDeletedTable(TableName tableName) throws IOException {
-    if (tableName.equals(TableName.META_TABLE_NAME))
+    if (tableName.equals(TableName.META_TABLE_NAME)) {
       return;
+    }
     MetaTableAccessor.deleteTableState(master.getConnection(), tableName);
   }
 
@@ -170,11 +181,17 @@ public class TableStateManager {
     return rv;
   }
 
+  public static class TableStateNotFoundException extends 
TableNotFoundException {
+    TableStateNotFoundException(TableName tableName) {
+      super(tableName.getNameAsString());
+    }
+  }
+
   @NonNull
   public TableState.State getTableState(TableName tableName) throws 
IOException {
     TableState currentState = readMetaState(tableName);
     if (currentState == null) {
-      throw new TableNotFoundException(tableName);
+      throw new TableStateNotFoundException(tableName);
     }
     return currentState.getState();
   }
@@ -194,42 +211,121 @@ public class TableStateManager {
 
   @Nullable
   protected TableState readMetaState(TableName tableName) throws IOException {
-    if (tableName.equals(TableName.META_TABLE_NAME)) {
-      return new TableState(tableName, TableState.State.ENABLED);
-    }
     return MetaTableAccessor.getTableState(master.getConnection(), tableName);
   }
 
   public void start() throws IOException {
     TableDescriptors tableDescriptors = master.getTableDescriptors();
+    migrateZooKeeper();
     Connection connection = master.getConnection();
     fixTableStates(tableDescriptors, connection);
   }
 
-  public static void fixTableStates(TableDescriptors tableDescriptors, 
Connection connection)
+  private void fixTableStates(TableDescriptors tableDescriptors, Connection 
connection)
       throws IOException {
-    final Map<String, TableDescriptor> allDescriptors =
-        tableDescriptors.getAllDescriptors();
+    final Map<String, TableDescriptor> allDescriptors = 
tableDescriptors.getAllDescriptors();
     final Map<String, TableState> states = new HashMap<>();
+    // NOTE: Ful hbase:meta table scan!
     MetaTableAccessor.fullScanTables(connection, new 
MetaTableAccessor.Visitor() {
       @Override
       public boolean visit(Result r) throws IOException {
         TableState state = MetaTableAccessor.getTableState(r);
-        if (state != null)
-          states.put(state.getTableName().getNameAsString(), state);
+        states.put(state.getTableName().getNameAsString(), state);
         return true;
       }
     });
-    for (Map.Entry<String, TableDescriptor> entry : allDescriptors.entrySet()) 
{
+    for (Map.Entry<String, TableDescriptor> entry: allDescriptors.entrySet()) {
       String table = entry.getKey();
       if (table.equals(TableName.META_TABLE_NAME.getNameAsString())) {
+        // This table is always enabled. No fixup needed. No entry in 
hbase:meta needed.
+        // Call through to fixTableState though in case a super class wants to 
do something.
+        fixTableState(new TableState(TableName.valueOf(table), 
TableState.State.ENABLED));
         continue;
       }
-      if (!states.containsKey(table)) {
-        LOG.warn(table + " has no state, assuming ENABLED");
+      TableState tableState = states.get(table);
+      if (tableState == null || tableState.getState() == null) {
+        LOG.warn(table + " has no table state in hbase:meta, assuming 
ENABLED");
         MetaTableAccessor.updateTableState(connection, 
TableName.valueOf(table),
             TableState.State.ENABLED);
+        fixTableState(new TableState(TableName.valueOf(table), 
TableState.State.ENABLED));
+      } else {
+        fixTableState(tableState);
       }
     }
   }
+
+  /**
+   * For subclasses in case they want to do fixup post hbase:meta.
+   */
+  protected void fixTableState(TableState tableState) throws IOException {}
+
+  /**
+   * This code is for case where a hbase2 Master is starting for the first 
time. ZooKeeper is
+   * where we used to keep table state. On first startup, read zookeeper and 
update hbase:meta
+   * with the table states found in zookeeper. This is tricky as we'll do this 
check every time we
+   * startup until mirroring is disabled. See the {@link 
#MIGRATE_TABLE_STATE_FROM_ZK_KEY} flag.
+   * Original form of this migration came in with HBASE-13032. It deleted all 
znodes when done.
+   * We can't do that if we want to support hbase-1.x clients who need to be 
able to read table
+   * state out of zk. See {@link MirroringTableStateManager}.
+   * @deprecated Since 2.0.0. Remove in hbase-3.0.0.
+   */
+  @Deprecated
+  private void migrateZooKeeper() throws IOException {
+    if 
(this.master.getConfiguration().getBoolean(MIGRATE_TABLE_STATE_FROM_ZK_KEY, 
false)) {
+      return;
+    }
+    try {
+      for (Map.Entry<TableName, TableState.State> entry:
+          
ZKDataMigrator.queryForTableStates(this.master.getZooKeeper()).entrySet()) {
+        if (this.master.getTableDescriptors().get(entry.getKey()) == null) {
+          deleteZooKeeper(entry.getKey());
+          LOG.info("Purged table state entry from zookeepr for table not in 
hbase:meta: " +
+              entry.getKey());
+          continue;
+        }
+        TableState.State state = null;
+        try {
+          state = getTableState(entry.getKey());
+        } catch (TableStateNotFoundException e) {
+          // This can happen; table exists but no TableState.
+        }
+        if (state == null) {
+          TableState.State zkstate = entry.getValue();
+          // Only migrate if it is an enable or disabled table. If in-between 
-- ENABLING or
+          // DISABLING then we have a problem; we are starting up an hbase-2 
on a cluster with
+          // RIT. It is going to be rough!
+          if (zkstate.equals(TableState.State.ENABLED) ||
+              zkstate.equals(TableState.State.DISABLED)) {
+            LOG.info("Migrating table state from zookeeper to hbase:meta; 
tableName=" +
+                entry.getKey() + ", state=" + entry.getValue());
+            updateMetaState(entry.getKey(), entry.getValue());
+          } else {
+            LOG.warn("Table={} has no state and zookeeper state is 
in-between={} (neither " +
+                "ENABLED or DISABLED); NOT MIGRATING table state", 
entry.getKey(), zkstate);
+          }
+        }
+        // What if the table states disagree? Defer to the hbase:meta setting 
rather than have the
+        // hbase-1.x support prevail.
+      }
+    } catch (KeeperException |InterruptedException e) {
+      LOG.warn("Failed reading table state from zookeeper", e);
+    }
+  }
+
+  /**
+   * Utility method that knows how to delete the old hbase-1.x table state 
znode.
+   * Used also by the Mirroring subclass.
+   * @deprecated Since 2.0.0. To be removed in hbase-3.0.0.
+   */
+  @Deprecated
+  protected void deleteZooKeeper(TableName tableName) {
+    try {
+      // Delete from ZooKeeper
+      String znode = 
ZNodePaths.joinZNode(this.master.getZooKeeper().getZNodePaths().tableZNode,
+          tableName.getNameAsString());
+      ZKUtil.deleteNodeFailSilent(this.master.getZooKeeper(), znode);
+    } catch (KeeperException e) {
+      LOG.warn("Failed deleting table state from zookeeper", e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/67b69fb2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index e09b29b..8b3dc61 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -1186,6 +1186,7 @@ public class AssignmentManager implements ServerListener {
     LOG.debug("Joining cluster...");
 
     // Scan hbase:meta to build list of existing regions, servers, and 
assignment
+    // hbase:meta is online when we get to here and TableStateManager has been 
started.
     loadMeta();
 
     for (int i = 0; master.getServerManager().countOfRegionServers() < 1; ++i) 
{
@@ -1232,6 +1233,10 @@ public class AssignmentManager implements ServerListener 
{
               regionStates.addRegionToServer(regionNode);
             } else if (localState == State.OFFLINE || regionInfo.isOffline()) {
               regionStates.addToOfflineRegions(regionNode);
+            } else if (localState == State.CLOSED && getTableStateManager().
+                isTableState(regionNode.getTable(), 
TableState.State.DISABLED)) {
+              // The region is CLOSED and the table is DISABLED, there is 
nothing to schedule;
+              // the region is inert.
             } else {
               // These regions should have a procedure in replay
               regionStates.addRegionInTransition(regionNode, null);
@@ -1496,8 +1501,10 @@ public class AssignmentManager implements ServerListener 
{
     synchronized (regionNode) {
       regionNode.transitionState(State.OPEN, 
RegionStates.STATES_EXPECTED_ON_OPEN);
       if (isMetaRegion(hri)) {
-        master.getTableStateManager().setTableState(TableName.META_TABLE_NAME,
-            TableState.State.ENABLED);
+        // Usually we'd set a table ENABLED at this stage but hbase:meta is 
ALWAYs enabled, it
+        // can't be disabled -- so skip the RPC (besides... enabled is managed 
by TableStateManager
+        // which is backed by hbase:meta... Avoid setting ENABLED to avoid 
having to update state
+        // on table that contains state.
         setMetaInitialized(hri, true);
       }
       regionStates.addRegionToServer(regionNode);

http://git-wip-us.apache.org/repos/asf/hbase/blob/67b69fb2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
index 151e3d6..c519fe6 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -37,7 +37,6 @@ 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.exceptions.HBaseException;
 import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
@@ -133,7 +132,7 @@ public class DeleteTableProcedure
         default:
           throw new UnsupportedOperationException("unhandled state=" + state);
       }
-    } catch (HBaseException|IOException e) {
+    } catch (IOException e) {
       if (isRollbackSupported(state)) {
         setFailure("master-delete-table", e);
       } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/67b69fb2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
index ae37a48..df0875e 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
@@ -27,7 +27,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CoordinatedStateException;
 import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
@@ -207,7 +206,7 @@ public final class ProcedureSyncWait {
   }
 
   protected static void waitRegionInTransition(final MasterProcedureEnv env,
-      final List<RegionInfo> regions) throws IOException, 
CoordinatedStateException {
+      final List<RegionInfo> regions) throws IOException {
     final RegionStates states = env.getAssignmentManager().getRegionStates();
     for (final RegionInfo region : regions) {
       ProcedureSyncWait.waitFor(env, "regions " + 
region.getRegionNameAsString() + " in transition",

http://git-wip-us.apache.org/repos/asf/hbase/blob/67b69fb2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
index 541fb8e..80cc5a8 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.exceptions.HBaseException;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.util.ModifyRegionUtils;
@@ -141,7 +140,7 @@ public class TruncateTableProcedure
         default:
           throw new UnsupportedOperationException("unhandled state=" + state);
       }
-    } catch (HBaseException|IOException e) {
+    } catch (IOException e) {
       if (isRollbackSupported(state)) {
         setFailure("master-truncate-table", e);
       } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/67b69fb2/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java
index b22b4ff..129341c 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ZKDataMigrator.java
@@ -36,13 +36,17 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * utlity method to migrate zookeeper data across HBase versions.
+ * Utlity method to migrate zookeeper data across HBase versions.
+ * @deprecated Since 2.0.0. To be removed in hbase-3.0.0.
  */
+@Deprecated
 @InterfaceAudience.Private
 public class ZKDataMigrator {
-
   private static final Logger LOG = 
LoggerFactory.getLogger(ZKDataMigrator.class);
 
+  // Shutdown constructor.
+  private ZKDataMigrator() {}
+
   /**
    * Method for table states migration.
    * Used when upgrading from pre-2.0 to 2.0
@@ -50,6 +54,7 @@ public class ZKDataMigrator {
    * and delete.
    * Used by master to clean migration from zk based states to
    * table descriptor based states.
+   * @deprecated Since 2.0.0. To be removed in hbase-3.0.0.
    */
   @Deprecated
   public static Map<TableName, TableState.State> queryForTableStates(ZKWatcher 
zkw)
@@ -90,6 +95,7 @@ public class ZKDataMigrator {
    * @param tableName table we're checking
    * @return Null or {@link ZooKeeperProtos.DeprecatedTableState.State} found 
in znode.
    * @throws KeeperException
+   * @deprecated Since 2.0.0. To be removed in hbase-3.0.0.
    */
   @Deprecated
   private static  ZooKeeperProtos.DeprecatedTableState.State getTableState(
@@ -113,5 +119,4 @@ public class ZKDataMigrator {
       throw ZKUtil.convert(e);
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/67b69fb2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
index 39c7967..83e69d2 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
@@ -27,7 +27,6 @@ import java.util.ArrayList;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.CoordinatedStateException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -276,7 +275,7 @@ public class TestMasterNoCluster {
 
       @Override
       void initializeZKBasedSystemTrackers() throws IOException, 
InterruptedException,
-          KeeperException, CoordinatedStateException, ReplicationException {
+          KeeperException, ReplicationException {
         super.initializeZKBasedSystemTrackers();
         // Record a newer server in server manager at first
         getServerManager().recordNewServerWithLock(newServer,

http://git-wip-us.apache.org/repos/asf/hbase/blob/67b69fb2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMirroringTableStateManager.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMirroringTableStateManager.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMirroringTableStateManager.java
new file mode 100644
index 0000000..d10c2b4
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMirroringTableStateManager.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master;
+
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.zookeeper.KeeperException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import java.io.IOException;
+
+import static junit.framework.TestCase.assertTrue;
+
+/**
+ * Tests that table state is mirrored out to zookeeper for hbase-1.x clients.
+ * Also tests that table state gets migrated from zookeeper on master start.
+ */
+@Category({ MasterTests.class, LargeTests.class })
+public class TestMirroringTableStateManager {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMirroringTableStateManager.class);
+  @Rule
+  public TestName name = new TestName();
+
+  private final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  @Before
+  public void before() throws Exception {
+    TEST_UTIL.startMiniCluster();
+  }
+
+  @After
+  public void after() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testMirroring() throws Exception {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY_STR);
+    ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
+    assertTrue(TableState.State.ENABLED.equals(getTableStateInZK(zkw, 
tableName)));
+    TEST_UTIL.getAdmin().disableTable(tableName);
+    assertTrue(TableState.State.DISABLED.equals(getTableStateInZK(zkw, 
tableName)));
+    TEST_UTIL.getAdmin().deleteTable(tableName);
+    assertTrue(getTableStateInZK(zkw, tableName) == null);
+  }
+
+  private TableState.State getTableStateInZK(ZKWatcher watcher, final 
TableName tableName)
+      throws KeeperException, IOException, InterruptedException {
+    String znode = ZNodePaths.joinZNode(watcher.znodePaths.tableZNode, 
tableName.getNameAsString());
+    byte [] data = ZKUtil.getData(watcher, znode);
+    if (data == null || data.length <= 0) {
+      return null;
+    }
+    try {
+      ProtobufUtil.expectPBMagicPrefix(data);
+      ZooKeeperProtos.DeprecatedTableState.Builder builder =
+          ZooKeeperProtos.DeprecatedTableState.newBuilder();
+      int magicLen = ProtobufUtil.lengthOfPBMagic();
+      ProtobufUtil.mergeFrom(builder, data, magicLen, data.length - magicLen);
+      return TableState.State.valueOf(builder.getState().toString());
+    } catch (IOException e) {
+      KeeperException ke = new KeeperException.DataInconsistencyException();
+      ke.initCause(e);
+      throw ke;
+    } catch (DeserializationException e) {
+      throw ZKUtil.convert(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/67b69fb2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java
index aa544aa..80b41a4 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableStateManager.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,27 +17,27 @@
  */
 package org.apache.hadoop.hbase.master;
 
-import java.io.IOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.zookeeper.KeeperException;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.apache.hadoop.hbase.util.Threads;
 import org.junit.After;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+
+import static junit.framework.TestCase.assertTrue;
 
 /**
  * Tests the default table lock manager
@@ -54,37 +54,41 @@ public class TestTableStateManager {
   @Rule
   public TestName name = new TestName();
 
+  @Before
+  public void before() throws Exception {
+    TEST_UTIL.startMiniCluster();
+  }
+
   @After
-  public void tearDown() throws Exception {
+  public void after() throws Exception {
     TEST_UTIL.shutdownMiniCluster();
   }
 
   @Test
-  public void testUpgradeFromZk() throws Exception {
+  public void testMigration() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    TEST_UTIL.startMiniCluster(2, 1);
-    TEST_UTIL.shutdownMiniHBaseCluster();
-    ZKWatcher watcher = TEST_UTIL.getZooKeeperWatcher();
-    setTableStateInZK(watcher, tableName, 
ZooKeeperProtos.DeprecatedTableState.State.DISABLED);
-    TEST_UTIL.restartHBaseCluster(1);
-
-    HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-    Assert.assertEquals(TableState.State.DISABLED,
-        master.getTableStateManager().getTableState(tableName));
-  }
-
-  private void setTableStateInZK(ZKWatcher watcher, final TableName tableName,
-                                 final 
ZooKeeperProtos.DeprecatedTableState.State state)
-      throws KeeperException, IOException {
-    String znode = ZNodePaths.joinZNode(watcher.znodePaths.tableZNode, 
tableName.getNameAsString());
-    if (ZKUtil.checkExists(watcher, znode) == -1) {
-      ZKUtil.createAndFailSilent(watcher, znode);
+    TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY_STR);
+    TEST_UTIL.getAdmin().disableTable(tableName);
+    // Table is disabled. Now remove the DISABLED column from the hbase:meta 
for this table's
+    // region. We want to see if Master will read the DISABLED from zk and 
make use of it as
+    // though it were reading the zk table state written by a hbase-1.x 
cluster.
+    TableState state = 
MetaTableAccessor.getTableState(TEST_UTIL.getConnection(), tableName);
+    assertTrue("State=" + state, 
state.getState().equals(TableState.State.DISABLED));
+    MetaTableAccessor.deleteTableState(TEST_UTIL.getConnection(), tableName);
+    assertTrue(MetaTableAccessor.getTableState(TEST_UTIL.getConnection(), 
tableName) == null);
+    // Now kill Master so a new one can come up and run through the zk 
migration.
+    HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
+    master.stop("Restarting");
+    while (!master.isStopped()) {
+      Threads.sleep(1);
     }
-    ZooKeeperProtos.DeprecatedTableState.Builder builder =
-        ZooKeeperProtos.DeprecatedTableState.newBuilder();
-    builder.setState(state);
-    byte[] data = ProtobufUtil.prependPBMagic(builder.build().toByteArray());
-    ZKUtil.setData(watcher, znode, data);
+    assertTrue(master.isStopped());
+    JVMClusterUtil.MasterThread newMasterThread = 
TEST_UTIL.getMiniHBaseCluster().startMaster();
+    master = newMasterThread.getMaster();
+    while (!master.isInitialized()) {
+      Threads.sleep(1);
+    }
+    assertTrue(MetaTableAccessor.getTableState(TEST_UTIL.getConnection(),
+        tableName).getState().equals(TableState.State.DISABLED));
   }
-
 }

Reply via email to